pg_walinspect - a new extension to get raw WAL data and WAL stats

Started by Bharath Rupireddyover 4 years ago90 messages
#1Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com

Hi,

While working on one of the internal features, we found that it is a
bit difficult to run pg_waldump for a normal user to know WAL info and
stats of a running postgres database instance in the cloud. Many a
times users or DBAs or developers would want to get and analyze
following:
1) raw WAL record associated with an LSN or raw WAL records between a
start LSN and end LSN for feeding to some other functionality
2) WAL statistics associated with an LSN or between start LSN and end
LSN for debugging or analytical purposes. The WAL stats are the number
of inserts, updates, deletes, index inserts, commits, checkpoints,
aborts, wal record sizes, FPI (Full Page Image) count etc. which are
basically everything that we get with pg_waldump --stats option plus
some other information as we may feel will be useful.

An available option is to use pg_waldump, a standalone program
emitting human readable WAL info into a standard output/file. This
works well when users have access to the system on which postgres is
running. But for a postgres database instance running in the cloud
environments, starting the pg_waldump, fetching and presenting its
output to the users in a structured way may be a bit hard to do.

How about we create a new extension, called pg_walinspect (synonymous
to pageinspect extension) with a bunch of SQL-callable functions that
get the raw WAL records or stats out of a running postgres database
instance in a more structured way that is easily consumable by all the
users or DBAs or developers? We can also provide these functionalities
into the core postgres (in xlogfuncs.c) instead of a new extension,
but we would like it to be pluggable so that the functions will be
used only if required.

[1]: a) bytea pg_get_wal_record(pg_lsn lsn); and bytea pg_get_wal_record(pg_lsn lsn, text wal_dir); - Returns a single row of raw WAL record of bytea type. WAL data is read from pg_wal or specified wal_dir directory.
extension can provide. These are not exhaustive; we can
add/remove/modify as we move further.

We would like to invite more thoughts from the hackers.

Credits: Thanks to Satya Narlapuram, Chen Liang (for some initial
work), Tianyu Zhang and Ashutosh Sharma (copied in cc) for internal
discussions.

[1]: a) bytea pg_get_wal_record(pg_lsn lsn); and bytea pg_get_wal_record(pg_lsn lsn, text wal_dir); - Returns a single row of raw WAL record of bytea type. WAL data is read from pg_wal or specified wal_dir directory.
a) bytea pg_get_wal_record(pg_lsn lsn); and bytea
pg_get_wal_record(pg_lsn lsn, text wal_dir); - Returns a single row of
raw WAL record of bytea type. WAL data is read from pg_wal or
specified wal_dir directory.

b) bytea[] pg_get_wal_record(pg_lsn start_lsn, pg_lsn end_lsn); and
bytea[] pg_get_wal_record(pg_lsn start_lsn, pg_lsn end_lsn, text
wal_dir); - Returns multiple rows of raw WAL records of bytea type,
one row per each WAL record. WAL data is read from pg_wal or specified
wal_dir directory.

CREATE TYPE walinspect_stats_type AS (stat1, stat2, stat3 …. statN);
c) walinspect_stats_type pg_get_wal_stats(pg_lsn lsn); and
walinspect_stats_type pg_get_wal_stats(pg_lsn lsn, text wal_dir); -
Returns a single row of WAL record’s stats of walinspect_stats_type
type. WAL data is read from pg_wal or specified wal_dir directory.

d) walinspect_stats_type[] pg_get_wal_stats(pg_lsn start_lsn, pg_lsn
end_lsn); and walinspect_stats_type[] pg_get_wal_stats(pg_lsn
start_lsn, pg_lsn end_lsn, text wal_dir); - Returns multiple rows of
WAL record stats of walinspect_stats_type type, one row per each WAL
record. WAL data is read from pg_wal or specified wal_dir directory.

e) walinspect_stats_type pg_get_wal_stats(bytea wal_record); -
Returns a single row of provided WAL record (wal_record) stats.

f) walinspect_stats_type pg_get_wal_stats_aggr(pg_lsn start_lsn,
pg_lsn end_lsn); and walinspect_stats_type
pg_get_wal_stats_aggr(pg_lsn start_lsn, pg_lsn end_lsn, text wal_dir);
- Returns a single row of aggregated stats of all the WAL records
between start_lsn and end_lsn. WAL data is read from pg_wal or
specified wal_dir directory.

CREATE TYPE walinspect_lsn_range_type AS (pg_lsn start_lsn, pg_lsn end_lsn);
g) walinspect_lsn_range_type walinspect_get_lsn_range(text
wal_dir); - Returns a single row of start LSN and end LSN of the WAL
records available under pg_wal or specified wal_dir directory.

Regards,
Bharath Rupireddy.

#2Bossart, Nathan
bossartn@amazon.com
In reply to: Bharath Rupireddy (#1)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On 9/8/21, 6:49 AM, "Bharath Rupireddy" <bharath.rupireddyforpostgres@gmail.com> wrote:

How about we create a new extension, called pg_walinspect (synonymous
to pageinspect extension) with a bunch of SQL-callable functions that
get the raw WAL records or stats out of a running postgres database
instance in a more structured way that is easily consumable by all the
users or DBAs or developers? We can also provide these functionalities
into the core postgres (in xlogfuncs.c) instead of a new extension,
but we would like it to be pluggable so that the functions will be
used only if required.

+1

Nathan

#3Michael Paquier
michael@paquier.xyz
In reply to: Bossart, Nathan (#2)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Sep 09, 2021 at 10:49:46PM +0000, Bossart, Nathan wrote:

+1

A backend approach has the advantage that you can use the proper locks
to make sure that a segment is not recycled or removed by a concurrent
checkpoint, so that would be reliable.
--
Michael

#4Drouvot, Bertrand
bdrouvot@amazon.com
In reply to: Bossart, Nathan (#2)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On 9/10/21 12:49 AM, Bossart, Nathan wrote:

On 9/8/21, 6:49 AM, "Bharath Rupireddy" <bharath.rupireddyforpostgres@gmail.com> wrote:

How about we create a new extension, called pg_walinspect (synonymous
to pageinspect extension) with a bunch of SQL-callable functions that
get the raw WAL records or stats out of a running postgres database
instance in a more structured way that is easily consumable by all the
users or DBAs or developers? We can also provide these functionalities
into the core postgres (in xlogfuncs.c) instead of a new extension,
but we would like it to be pluggable so that the functions will be
used only if required.

+1

Nathan

+1

Bertrand

#5Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Michael Paquier (#3)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Sep 10, 2021 at 7:21 AM Michael Paquier <michael@paquier.xyz> wrote:

On Thu, Sep 09, 2021 at 10:49:46PM +0000, Bossart, Nathan wrote:

+1

A backend approach has the advantage that you can use the proper locks
to make sure that a segment is not recycled or removed by a concurrent
checkpoint, so that would be reliable.

Thanks for sharing your thoughts. IMO, using locks for showing WAL
stats isn't a good way, because these new functions may block the
checkpointer from removing/recycling the WAL files. We don't want to
do that. If at all, user has asked stats of an LSN/range of LSNs if
it is/they are available in the pg_wal directory, we provide the info
otherwise we can throw warnings/errors. This behaviour is pretty much
in sycn with what pg_waldump does right now.

And, some users may not need these new functions at all, so in such
cases going with an extension way makes it more usable.

Regards,
Bharath Rupireddy.

#6Bruce Momjian
bruce@momjian.us
In reply to: Bharath Rupireddy (#1)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Sep 8, 2021 at 07:18:08PM +0530, Bharath Rupireddy wrote:

Hi,

While working on one of the internal features, we found that it is a
bit difficult to run pg_waldump for a normal user to know WAL info and
stats of a running postgres database instance in the cloud. Many a
times users or DBAs or developers would want to get and analyze
following:

Uh, are we going to implement everything that is only available at the
command line as an extension just for people who are using managed cloud
services where the command line is not available and the cloud provider
has not made that information accessible? Seems like this might lead to
a lot of duplicated effort.

--
Bruce Momjian <bruce@momjian.us> https://momjian.us
EDB https://enterprisedb.com

If only the physical world exists, free will is an illusion.

#7Chapman Flack
chap@anastigmatix.net
In reply to: Bruce Momjian (#6)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On 10/05/21 18:07, Bruce Momjian wrote:

Uh, are we going to implement everything that is only available at the
command line as an extension just for people who are using managed cloud
services

One extension that runs a curated menu of command-line tools for you
and returns their stdout?

Regards,
-Chap

#8Jeremy Schneider
schneider@ardentperf.com
In reply to: Bruce Momjian (#6)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On 10/5/21 15:07, Bruce Momjian wrote:

On Wed, Sep 8, 2021 at 07:18:08PM +0530, Bharath Rupireddy wrote:

While working on one of the internal features, we found that it is a
bit difficult to run pg_waldump for a normal user to know WAL info and
stats of a running postgres database instance in the cloud. Many a
times users or DBAs or developers would want to get and analyze
following:

Uh, are we going to implement everything that is only available at the
command line as an extension just for people who are using managed cloud
services where the command line is not available and the cloud provider
has not made that information accessible? Seems like this might lead to
a lot of duplicated effort.

No. For most command line utilities, there's no reason to expose them in
SQL or they already are exposed in SQL. For example, everything in
pg_controldata is already available via SQL functions.

Specifically exposing pg_waldump functionality in SQL could speed up
finding bugs in the PG logical replication code. We found and fixed a
few over this past year, but there are more lurking out there.

Having the extension in core is actually the only way to avoid
duplicated effort, by having shared code which the pg_dump binary and
the extension both wrap or call.

-Jeremy

--
http://about.me/jeremy_schneider

#9Bruce Momjian
bruce@momjian.us
In reply to: Chapman Flack (#7)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Tue, Oct 5, 2021 at 06:22:19PM -0400, Chapman Flack wrote:

On 10/05/21 18:07, Bruce Momjian wrote:

Uh, are we going to implement everything that is only available at the
command line as an extension just for people who are using managed cloud
services

One extension that runs a curated menu of command-line tools for you
and returns their stdout?

Yes, that would make sense, and something the cloud service providers
would write.

--
Bruce Momjian <bruce@momjian.us> https://momjian.us
EDB https://enterprisedb.com

If only the physical world exists, free will is an illusion.

#10Bruce Momjian
bruce@momjian.us
In reply to: Jeremy Schneider (#8)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Tue, Oct 5, 2021 at 03:30:07PM -0700, Jeremy Schneider wrote:

On 10/5/21 15:07, Bruce Momjian wrote:

On Wed, Sep 8, 2021 at 07:18:08PM +0530, Bharath Rupireddy wrote:

While working on one of the internal features, we found that it is a
bit difficult to run pg_waldump for a normal user to know WAL info and
stats of a running postgres database instance in the cloud. Many a
times users or DBAs or developers would want to get and analyze
following:

Uh, are we going to implement everything that is only available at the
command line as an extension just for people who are using managed cloud
services where the command line is not available and the cloud provider
has not made that information accessible? Seems like this might lead to
a lot of duplicated effort.

No. For most command line utilities, there's no reason to expose them in
SQL or they already are exposed in SQL. For example, everything in
pg_controldata is already available via SQL functions.

That's a good example.

Specifically exposing pg_waldump functionality in SQL could speed up
finding bugs in the PG logical replication code. We found and fixed a
few over this past year, but there are more lurking out there.

Uh, why is pg_waldump more important than other command line tool
information?

Having the extension in core is actually the only way to avoid
duplicated effort, by having shared code which the pg_dump binary and
the extension both wrap or call.

Uh, aren't you duplicating code by having pg_waldump as a command-line
tool and an extension?

--
Bruce Momjian <bruce@momjian.us> https://momjian.us
EDB https://enterprisedb.com

If only the physical world exists, free will is an illusion.

#11Jeremy Schneider
schneider@ardentperf.com
In reply to: Bruce Momjian (#10)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On 10/5/21 17:43, Bruce Momjian wrote:

On Tue, Oct 5, 2021 at 03:30:07PM -0700, Jeremy Schneider wrote:

Specifically exposing pg_waldump functionality in SQL could speed up
finding bugs in the PG logical replication code. We found and fixed a
few over this past year, but there are more lurking out there.

Uh, why is pg_waldump more important than other command line tool
information?

Going down the list of all other utilities in src/bin:

* pg_amcheck, pg_config, pg_controldata: already available in SQL
* psql, pgbench, pg_dump: already available as client-side apps
* initdb, pg_archivecleanup, pg_basebackup, pg_checksums, pg_ctl,
pg_resetwal, pg_rewind, pg_upgrade, pg_verifybackup: can't think of any
possible use case outside server OS access, most of these are too low
level and don't even make sense in SQL
* pg_test_fsync, pg_test_timing: marginally interesting ideas in SQL,
don't feel any deep interest myself

Speaking selfishly, there are a few reasons I would be specifically
interested in pg_waldump (the only remaining one on the list).

.

First, to better support existing features around logical replication
and decoding.

In particular, it seems inconsistent to me that all the replication
management SQL functions take LSNs as arguments - and yet there's no
SQL-based way to find the LSNs that you are supposed to pass into these
functions.

https://www.postgresql.org/docs/current/functions-admin.html#FUNCTIONS-REPLICATION

Over the past few years I've been pulled in to help several large PG
users who ran into these bugs, and it's very painful - because the only
real remediation is to drop and recreate the replication slot, which
means either re-copying all the data to the downstream system or
figuring out a way to resync it. Some PG users have 3rd party tools like
HVR which can do row-by-row resync (IIUC), but no matter how you slice
it, we're talking about a lot of pain for people replicating large data
sets between multiple systems. In most cases, the only/best option even
with very large tables is to just make a fresh copy of the data - which
can translate to a business outage of hours or even days.

My favorite example is the SQL function pg_replication_slot_advance() -
this could really help PG users find less painful solutions to broken
decoding, however it's not really possible to /know/ an LSN to advance
to without inspecting WAL. ISTM there's a strong use case here for a SQL
interface on WAL inspection.

.

Second: debugging and troubleshooting logical replication and decoding bugs.

I helped track down a few logical replication bugs and get fixed into
code at postgresql.org this past year. (But I give credit to others who
are much better at C than I am, and who did a lot more work than I did
on these bugs!)

Logical decoding bugs are some of the hardest to fix - because all you
have is a WAL stream, but you don't know the SQL or workload patterns or
PG code paths which created that WAL stream.

Dumping the WAL - knowing which objects and which types of operations
are involved and stats like number of changes or number of
subtransactions - this helps identify which transaction and what SQL in
the application triggered the failure, which can help find short-term
workarounds. Businesses need those short-term workarounds so they can
keep going while we work on finding and fixing bugs, which can take some
time. This is another place where I think a SQL interface to WAL would
be helpful to PG users. Especially the ability to filter and trace a
single transaction through a large number of WAL files in the directory.

.

Third: statistics on WAL - especially full page writes. Giving users the
full power of SQL allows much more sophisticated analysis of the WAL
records. Personally, I'd probably find myself importing all the WAL
stats into the DB anyway because SQL is my preferred data manipulation
language.

Having the extension in core is actually the only way to avoid
duplicated effort, by having shared code which the pg_dump binary and
the extension both wrap or call.

Uh, aren't you duplicating code by having pg_waldump as a command-line
tool and an extension?

Well this whole conversation is just theoretical anyway until the code
is shared. :) But if Bharath is writing functions to decode WAL, then
wouldn't we just have pg_waldump use these same functions in order to
avoid duplicating code?

Bharath - was some code already posted and I just missed it?

Looking at the proposed API from the initial email, I like that there's
both stats functionality and WAL record inspection functionality
(similar to pg_waldump). I like that there's the ability to pull the
records as raw bytea data, however I think we're also going to want an
ability in v1 of the patch to decode it (similar to pageinspect
heap_page_item_attrs, etc).

Another feature that might be interesting down the road would be the
ability to provide filtering of WAL records for security purposes. For
example, allowing a user to only dump raw WAL records for one particular
database, or maybe excluding WAL records that change system catalogs or
the like. But I probably wouldn't start here, personally.

Now then.... as Blaise Pascal said in 1657 (and as was also said by
Winston Churchill, Mark Twain, etc).... "I'm sorry I wrote you such a
long letter; I didn't have time to write a short one."

-Jeremy

--
http://about.me/jeremy_schneider

#12Bruce Momjian
bruce@momjian.us
In reply to: Jeremy Schneider (#11)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Oct 6, 2021 at 09:56:34AM -0700, Jeremy Schneider wrote:

On 10/5/21 17:43, Bruce Momjian wrote:

On Tue, Oct 5, 2021 at 03:30:07PM -0700, Jeremy Schneider wrote:

Specifically exposing pg_waldump functionality in SQL could speed up
finding bugs in the PG logical replication code. We found and fixed a
few over this past year, but there are more lurking out there.

Uh, why is pg_waldump more important than other command line tool
information?

Going down the list of all other utilities in src/bin:

* pg_amcheck, pg_config, pg_controldata: already available in SQL
* psql, pgbench, pg_dump: already available as client-side apps
* initdb, pg_archivecleanup, pg_basebackup, pg_checksums, pg_ctl,
pg_resetwal, pg_rewind, pg_upgrade, pg_verifybackup: can't think of any
possible use case outside server OS access, most of these are too low
level and don't even make sense in SQL
* pg_test_fsync, pg_test_timing: marginally interesting ideas in SQL,
don't feel any deep interest myself

Speaking selfishly, there are a few reasons I would be specifically
interested in pg_waldump (the only remaining one on the list).

This is the analysis I was looking for to understand if copying the
features of command-line tools in extensions was a wise direction.

--
Bruce Momjian <bruce@momjian.us> https://momjian.us
EDB https://enterprisedb.com

If only the physical world exists, free will is an illusion.

#13Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Jeremy Schneider (#11)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On 2021-Oct-06, Jeremy Schneider wrote:

Well this whole conversation is just theoretical anyway until the code
is shared. :) But if Bharath is writing functions to decode WAL, then
wouldn't we just have pg_waldump use these same functions in order to
avoid duplicating code?

Actually, a lot of the code is already shared, since the rmgrdesc
routines are in src/backend. Keep in mind that it was there before
pg_xlogdump existed, to support WAL_DEBUG. When pg_xlogdump was added,
what we did was allow that backend-only code be compilable in a frontend
environment. Also, we already have xlogreader.

So pg_waldump itself is mostly scaffolding to let the frontend
environment get argument values to pass to backend-enabled code. The
only really interesting, novel thing is the --stats mode ... and I bet
you can write that with some SQL-level aggregation of the raw data, no
need for any C code.

--
Álvaro Herrera Valdivia, Chile — https://www.EnterpriseDB.com/

#14Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Jeremy Schneider (#11)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Oct 6, 2021 at 10:26 PM Jeremy Schneider
<schneider@ardentperf.com> wrote:

On 10/5/21 17:43, Bruce Momjian wrote:

On Tue, Oct 5, 2021 at 03:30:07PM -0700, Jeremy Schneider wrote:

Specifically exposing pg_waldump functionality in SQL could speed up
finding bugs in the PG logical replication code. We found and fixed a
few over this past year, but there are more lurking out there.

Uh, why is pg_waldump more important than other command line tool
information?

Going down the list of all other utilities in src/bin:

* pg_amcheck, pg_config, pg_controldata: already available in SQL
* psql, pgbench, pg_dump: already available as client-side apps
* initdb, pg_archivecleanup, pg_basebackup, pg_checksums, pg_ctl,
pg_resetwal, pg_rewind, pg_upgrade, pg_verifybackup: can't think of any
possible use case outside server OS access, most of these are too low
level and don't even make sense in SQL
* pg_test_fsync, pg_test_timing: marginally interesting ideas in SQL,
don't feel any deep interest myself

Speaking selfishly, there are a few reasons I would be specifically
interested in pg_waldump (the only remaining one on the list).

Thanks Jeremy for the analysis.

First, to better support existing features around logical replication
and decoding.

In particular, it seems inconsistent to me that all the replication
management SQL functions take LSNs as arguments - and yet there's no
SQL-based way to find the LSNs that you are supposed to pass into these
functions.

https://www.postgresql.org/docs/current/functions-admin.html#FUNCTIONS-REPLICATION

Over the past few years I've been pulled in to help several large PG
users who ran into these bugs, and it's very painful - because the only
real remediation is to drop and recreate the replication slot, which
means either re-copying all the data to the downstream system or
figuring out a way to resync it. Some PG users have 3rd party tools like
HVR which can do row-by-row resync (IIUC), but no matter how you slice
it, we're talking about a lot of pain for people replicating large data
sets between multiple systems. In most cases, the only/best option even
with very large tables is to just make a fresh copy of the data - which
can translate to a business outage of hours or even days.

My favorite example is the SQL function pg_replication_slot_advance() -
this could really help PG users find less painful solutions to broken
decoding, however it's not really possible to /know/ an LSN to advance
to without inspecting WAL. ISTM there's a strong use case here for a SQL
interface on WAL inspection.

Second: debugging and troubleshooting logical replication and decoding bugs.

I helped track down a few logical replication bugs and get fixed into
code at postgresql.org this past year. (But I give credit to others who
are much better at C than I am, and who did a lot more work than I did
on these bugs!)

Logical decoding bugs are some of the hardest to fix - because all you
have is a WAL stream, but you don't know the SQL or workload patterns or
PG code paths which created that WAL stream.

Dumping the WAL - knowing which objects and which types of operations
are involved and stats like number of changes or number of
subtransactions - this helps identify which transaction and what SQL in
the application triggered the failure, which can help find short-term
workarounds. Businesses need those short-term workarounds so they can
keep going while we work on finding and fixing bugs, which can take some
time. This is another place where I think a SQL interface to WAL would
be helpful to PG users. Especially the ability to filter and trace a
single transaction through a large number of WAL files in the directory.

Third: statistics on WAL - especially full page writes. Giving users the
full power of SQL allows much more sophisticated analysis of the WAL
records. Personally, I'd probably find myself importing all the WAL
stats into the DB anyway because SQL is my preferred data manipulation
language.

Just to add to the above points, with the new extension pg_walinspect
we will have following advantages:
1) Usability - SQL callable functions will be easier to use for the
users/admins/developers.
2) Access Control - we can provide better access control for the WAL data/stats.
3) Emitting the actual WAL data(as bytea structure) and stats via SQL
callable functions will help to analyze and answer questions like how
much WAL data is being generated in the system, what kind of WAL data
it is, how many FPWs are happening and so on. Jermey has already given
more realistic use cases.
4) I came across this - there's a similar capability in SQL server -
https://www.mssqltips.com/sqlservertip/3076/how-to-read-the-sql-server-database-transaction-log/

Having the extension in core is actually the only way to avoid
duplicated effort, by having shared code which the pg_dump binary and
the extension both wrap or call.

Uh, aren't you duplicating code by having pg_waldump as a command-line
tool and an extension?

Well this whole conversation is just theoretical anyway until the code
is shared. :) But if Bharath is writing functions to decode WAL, then
wouldn't we just have pg_waldump use these same functions in order to
avoid duplicating code?

Bharath - was some code already posted and I just missed it?

Looking at the proposed API from the initial email, I like that there's
both stats functionality and WAL record inspection functionality
(similar to pg_waldump). I like that there's the ability to pull the
records as raw bytea data, however I think we're also going to want an
ability in v1 of the patch to decode it (similar to pageinspect
heap_page_item_attrs, etc).

I'm yet to start working on the patch. I will be doing it soon.

Another feature that might be interesting down the road would be the
ability to provide filtering of WAL records for security purposes. For
example, allowing a user to only dump raw WAL records for one particular
database, or maybe excluding WAL records that change system catalogs or
the like. But I probably wouldn't start here, personally.

+1.

Regards,
Bharath Rupireddy.

#15Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Bharath Rupireddy (#14)
2 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Oct 7, 2021 at 10:43 AM Bharath Rupireddy <
bharath.rupireddyforpostgres@gmail.com> wrote:

Looking at the proposed API from the initial email, I like that there's
both stats functionality and WAL record inspection functionality
(similar to pg_waldump). I like that there's the ability to pull the
records as raw bytea data, however I think we're also going to want an
ability in v1 of the patch to decode it (similar to pageinspect
heap_page_item_attrs, etc).

I'm yet to start working on the patch. I will be doing it soon.

Thanks all. Here's the v1 patch set for the new extension pg_walinspect.
Note that I didn't include the documentation part now, I will be doing it a
bit later.

Please feel free to review and provide your thoughts.

Regards,
Bharath Rupireddy.

Attachments:

v1-0001-pg_walinspect.patchapplication/x-patch; name=v1-0001-pg_walinspect.patchDownload
From 712077e3ea86790e887f72ec3546a8661a62389a Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 17 Nov 2021 13:43:41 +0000
Subject: [PATCH v1] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 106 +++
 contrib/pg_walinspect/pg_walinspect.c        | 801 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  14 +-
 src/include/access/xlogreader.h              |   2 -
 8 files changed, 950 insertions(+), 9 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 87bf87ab90..780059ce66 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -39,6 +39,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..bd641abd04
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,106 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+--
+-- pg_get_first_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_valid_wal_record_lsn(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_valid_wal_record_lsn'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) TO pg_monitor;
+
+--
+-- pg_verify_raw_wal_record()
+--
+CREATE FUNCTION pg_verify_raw_wal_record(IN record bytea,
+    OUT is_valid boolean
+)
+AS 'MODULE_PATHNAME', 'pg_verify_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_verify_raw_wal_record(bytea) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_verify_raw_wal_record(bytea) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT rmgr text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info_2()
+--
+CREATE FUNCTION pg_get_wal_record_info_2(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT rmgr text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info_2'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info_2(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info_2(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT rmgr text,
+    OUT count int8,
+    OUT count_per float4,
+    OUT record_size int8,
+    OUT record_size_per float4,
+    OUT fpi_size int8,
+    OUT fpi_size_per float4,
+    OUT combined_size int8,
+    OUT combined_size_per float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..01c8c49ea7
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,801 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Portions Copyright (c) 2012-2021, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+PG_MODULE_MAGIC;
+
+typedef struct Stats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+} Stats;
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	Stats		rmgr_stats[RM_NEXT_ID];
+	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+}			XLogRecStats;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info_2);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+
+static bool IsXLogRecordValid(XLogRecord *record);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record,
+											bool warning);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls);
+static void StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record);
+static void GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+								TupleDesc tupdesc, Datum *values, bool *nulls);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len, uint64 fpi_len,
+							 uint64 total_fpi_len, uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls);
+static void GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+								XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Intialize WAL reader and identify the first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record, bool warning)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (!xlogreader)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	/*
+	 * Display a message that we're skipping data if the given lsn wasn't a
+	 * pointer to the start of a record and also wasn't a pointer to the
+	 * beginning of a segment (e.g. we were used in file mode).
+	 */
+	if (warning && *first_record != lsn &&
+		XLogSegmentOffset(lsn, wal_segment_size) != 0)
+		ereport(WARNING,
+				(errmsg_plural("first record is after %X/%X, at %X/%X, skipping over %u byte",
+							   "first record is after %X/%X, at %X/%X, skipping over %u bytes",
+							   (*first_record - lsn),
+							   LSN_FORMAT_ARGS(lsn),
+							   LSN_FORMAT_ARGS(*first_record),
+							   (uint32) (*first_record - lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read the WAL next record
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	char	   *errormsg;
+	XLogRecord *record;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (!record)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get the raw WAL record. This function will wait for the required WAL at LSN.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	   *raw_record;
+	uint32		rec_len;
+	char	   *raw_record_data;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	Assert(record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	/*
+	 * XXX: Check the authenticity of the WAL record here? Although it might
+	 * have been done by XLogReadRecord, after memcpy, just check for
+	 * raw_record_data correctness.
+	 */
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[0] = LSNGetDatum(first_record);
+	values[1] = PointerGetDatum(raw_record);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_COLS
+}
+
+/*
+ * Get the raw WAL record. This function will wait for the required WAL at LSN.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	XLogReaderFree(xlogreader);
+
+	PG_RETURN_LSN(first_record);
+}
+
+/*
+ * Same as ValidXLogRecord but without XLogReaderState and XLogRecPtr.
+ */
+static bool
+IsXLogRecordValid(XLogRecord *record)
+{
+	pg_crc32c	crc;
+
+	/* Calculate the CRC. */
+	INIT_CRC32C(crc);
+	COMP_CRC32C(crc, ((char *) record) + SizeOfXLogRecord, record->xl_tot_len - SizeOfXLogRecord);
+	/* Include the record header last. */
+	COMP_CRC32C(crc, (char *) record, offsetof(XLogRecord, xl_crc));
+	FIN_CRC32C(crc);
+
+	if (!EQ_CRC32C(record->xl_crc, crc))
+		return false;
+
+	return true;
+}
+
+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{
+	bytea	   *raw_record;
+	XLogRecord *record;
+	bool		valid;
+
+	raw_record = PG_GETARG_BYTEA_PP(0);
+	record = (XLogRecord *) VARDATA_ANY(raw_record);
+	valid = IsXLogRecordValid(record);
+
+	PG_RETURN_BOOL(valid);
+}
+
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+static void
+GetXLogRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
+{
+	int			block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data. It doesn't seem worth it to
+	 * add an accessor macro for this.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += record->blocks[block_id].bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Get the WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32		rec_len;
+	uint32		fpi_len;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int			block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	   *data;
+	char	   *main_data;
+	uint32		main_data_len;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	values[0] = LSNGetDatum(lsn);
+	values[1] = LSNGetDatum(XLogRecGetPrev(record));
+	values[2] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[3] = CStringGetTextDatum(desc->rm_name);
+	values[4] = UInt32GetDatum(rec_len);
+	values[5] = UInt32GetDatum(XLogRecGetTotalLen(record));
+
+	initStringInfo(&rec_desc);
+
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "desc: UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "desc: %s ", id);
+
+	initStringInfo(&temp);
+
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+
+	values[6] = CStringGetTextDatum(rec_desc.data);
+
+	pfree(temp.data);
+
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+		appendStringInfo(&rec_blk_ref, "blkref #%u: rel %u/%u/%u fork %s blk %u",
+						 block_id, rnode.spcNode, rnode.dbNode,
+						 rnode.relNode, forkNames[forknum], blk);
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	values[7] = CStringGetTextDatum(rec_blk_ref.data);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[8] = PointerGetDatum(data);
+	values[9] = UInt32GetDatum(main_data_len);
+}
+
+/*
+ * Get the WAL record info and data.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	Assert(record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get the WAL record info and data between start LSN and end LSN.
+ */
+Datum
+pg_get_wal_record_info_2(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	if (XLogRecPtrIsInvalid(start_lsn) || XLogRecPtrIsInvalid(end_lsn))
+		PG_RETURN_NULL();
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		Assert(record);
+
+		if (xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls);
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+
+	PG_RETURN_VOID();
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+static void
+StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record)
+{
+	RmgrId		rmid;
+	uint8		recid;
+	uint32		rec_len;
+	uint32		fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics. */
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
+/*
+ * Display a single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls)
+{
+	double		n_pct;
+	double		rec_len_pct;
+	double		fpi_len_pct;
+	double		tot_len_pct;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[0] = CStringGetTextDatum(name);
+	values[1] = Int64GetDatum(n);
+	values[2] = Float4GetDatum(n_pct);
+	values[3] = Int64GetDatum(rec_len);
+	values[4] = Float4GetDatum(rec_len_pct);
+	values[5] = Int64GetDatum(fpi_len);
+	values[6] = Float4GetDatum(fpi_len_pct);
+	values[7] = Int64GetDatum(tot_len);
+	values[8] = Float4GetDatum(tot_len_pct);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+					TupleDesc tupdesc, Datum *values, bool *nulls)
+{
+	int			ri;
+	uint64		total_count = 0;
+	uint64		total_rec_len = 0;
+	uint64		total_fpi_len = 0;
+	uint64		total_len = 0;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+}
+
+/*
+ * Get the WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+					XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogRecStats stats;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_STATS_COLS];
+	bool		nulls[PG_GET_WAL_STATS_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		Assert(record);
+
+		if (xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		StoreXLogRecordStats(&stats, xlogreader);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, tupstore, tupdesc, values, nulls);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get the WAL stats between start LSN and end LSN.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	if (XLogRecPtrIsInvalid(start_lsn) || XLogRecPtrIsInvalid(end_lsn))
+		PG_RETURN_NULL();
+
+	GetWalStatsInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index f39f8044a9..6be533d501 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -956,13 +956,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1080,6 +1073,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index de6fd791fe..e6f98cbb6f 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
-- 
2.25.1

v1-0002-pg_walinspect-tests.patchapplication/x-patch; name=v1-0002-pg_walinspect-tests.patchDownload
From a9d061ca82a49fbe8044de1925247f300149a71e Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 17 Nov 2021 13:31:31 +0000
Subject: [PATCH v1] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 37 +++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 33 +++++++++++++++++
 2 files changed, 70 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..f15ec0e5e8
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,37 @@
+CREATE EXTENSION pg_walinspect;
+SELECT pg_current_wal_lsn() AS wal_lsn \gset
+CREATE TABLE sample_tbl(col1 int, col2 int);
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+CHECKPOINT;
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn') AS valid_wal_lsn \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+SELECT record AS raw_wal_rec FROM pg_get_raw_wal_record(:'valid_wal_lsn') \gset
+SELECT * FROM pg_verify_raw_wal_record(:'raw_wal_rec');
+ is_valid 
+----------
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+CHECKPOINT;
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn2') AS valid_wal_lsn2 \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info_2(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..ac0404bb90
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,33 @@
+CREATE EXTENSION pg_walinspect;
+
+SELECT pg_current_wal_lsn() AS wal_lsn \gset
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+
+CHECKPOINT;
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn');
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn') AS valid_wal_lsn \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn');
+
+SELECT record AS raw_wal_rec FROM pg_get_raw_wal_record(:'valid_wal_lsn') \gset
+
+SELECT * FROM pg_verify_raw_wal_record(:'raw_wal_rec');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn');
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+
+CHECKPOINT;
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn2') AS valid_wal_lsn2 \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info_2(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
-- 
2.25.1

#16Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Bharath Rupireddy (#15)
2 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Nov 18, 2021 at 6:43 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

On Thu, Oct 7, 2021 at 10:43 AM Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com> wrote:

Looking at the proposed API from the initial email, I like that there's
both stats functionality and WAL record inspection functionality
(similar to pg_waldump). I like that there's the ability to pull the
records as raw bytea data, however I think we're also going to want an
ability in v1 of the patch to decode it (similar to pageinspect
heap_page_item_attrs, etc).

I'm yet to start working on the patch. I will be doing it soon.

Thanks all. Here's the v1 patch set for the new extension pg_walinspect. Note that I didn't include the documentation part now, I will be doing it a bit later.

Please feel free to review and provide your thoughts.

The v1 patch set was failing to compile on Windows. Here's the v2
patch set fixing that.

Regards,
Bharath Rupireddy.

Attachments:

v2-0001-pg_walinspect.patchapplication/octet-stream; name=v2-0001-pg_walinspect.patchDownload
From b6ccace4c99098f87f0efc487587fe9c4c2d0117 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforwork@gmail.com>
Date: Thu, 25 Nov 2021 15:41:47 +0530
Subject: [PATCH v2] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 106 +++
 contrib/pg_walinspect/pg_walinspect.c        | 779 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  35 +-
 src/common/relpath.c                         |   6 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   4 +-
 src/include/common/relpath.h                 |   1 +
 12 files changed, 960 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 87bf87ab90..780059ce66 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -39,6 +39,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..bd641abd04
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,106 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+--
+-- pg_get_first_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_valid_wal_record_lsn(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_valid_wal_record_lsn'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) TO pg_monitor;
+
+--
+-- pg_verify_raw_wal_record()
+--
+CREATE FUNCTION pg_verify_raw_wal_record(IN record bytea,
+    OUT is_valid boolean
+)
+AS 'MODULE_PATHNAME', 'pg_verify_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_verify_raw_wal_record(bytea) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_verify_raw_wal_record(bytea) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT rmgr text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info_2()
+--
+CREATE FUNCTION pg_get_wal_record_info_2(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT rmgr text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info_2'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info_2(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info_2(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT rmgr text,
+    OUT count int8,
+    OUT count_per float4,
+    OUT record_size int8,
+    OUT record_size_per float4,
+    OUT fpi_size int8,
+    OUT fpi_size_per float4,
+    OUT combined_size int8,
+    OUT combined_size_per float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..c38bd4b95a
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,779 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Portions Copyright (c) 2012-2021, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+PG_MODULE_MAGIC;
+
+typedef struct Stats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+} Stats;
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	Stats		rmgr_stats[RM_NEXT_ID];
+	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+}			XLogRecStats;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info_2);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record,
+											bool warning);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls);
+static void StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record);
+static void GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+								TupleDesc tupdesc, Datum *values, bool *nulls);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len, uint64 fpi_len,
+							 uint64 total_fpi_len, uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls);
+static void GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+								XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Intialize WAL reader and identify the first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record, bool warning)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (!xlogreader)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	/*
+	 * Display a message that we're skipping data if the given lsn wasn't a
+	 * pointer to the start of a record and also wasn't a pointer to the
+	 * beginning of a segment (e.g. we were used in file mode).
+	 */
+	if (warning && *first_record != lsn &&
+		XLogSegmentOffset(lsn, wal_segment_size) != 0)
+		ereport(WARNING,
+				(errmsg_plural("first record is after %X/%X, at %X/%X, skipping over %u byte",
+							   "first record is after %X/%X, at %X/%X, skipping over %u bytes",
+							   (*first_record - lsn),
+							   LSN_FORMAT_ARGS(lsn),
+							   LSN_FORMAT_ARGS(*first_record),
+							   (uint32) (*first_record - lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read the WAL next record
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	char	   *errormsg;
+	XLogRecord *record;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (!record)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get the raw WAL record. This function will wait for the required WAL at LSN.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	   *raw_record;
+	uint32		rec_len;
+	char	   *raw_record_data;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	Assert(record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	/*
+	 * XXX: Check the authenticity of the WAL record here? Although it might
+	 * have been done by XLogReadRecord, after memcpy, just check for
+	 * raw_record_data correctness.
+	 */
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[0] = LSNGetDatum(first_record);
+	values[1] = PointerGetDatum(raw_record);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_COLS
+}
+
+/*
+ * Get the raw WAL record. This function will wait for the required WAL at LSN.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	XLogReaderFree(xlogreader);
+
+	PG_RETURN_LSN(first_record);
+}
+
+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{
+	bytea	   *raw_record;
+	XLogRecord *record;
+	bool		valid;
+
+	raw_record = PG_GETARG_BYTEA_PP(0);
+	record = (XLogRecord *) VARDATA_ANY(raw_record);
+	valid = IsXLogRecordValid(record);
+
+	PG_RETURN_BOOL(valid);
+}
+
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+static void
+GetXLogRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
+{
+	int			block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data. It doesn't seem worth it to
+	 * add an accessor macro for this.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += record->blocks[block_id].bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Get the WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32		rec_len;
+	uint32		fpi_len;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int			block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	   *data;
+	char	   *main_data;
+	uint32		main_data_len;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	values[0] = LSNGetDatum(lsn);
+	values[1] = LSNGetDatum(XLogRecGetPrev(record));
+	values[2] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[3] = CStringGetTextDatum(desc->rm_name);
+	values[4] = UInt32GetDatum(rec_len);
+	values[5] = UInt32GetDatum(XLogRecGetTotalLen(record));
+
+	initStringInfo(&rec_desc);
+
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "desc: UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "desc: %s ", id);
+
+	initStringInfo(&temp);
+
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+
+	values[6] = CStringGetTextDatum(rec_desc.data);
+
+	pfree(temp.data);
+
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+		appendStringInfo(&rec_blk_ref, "blkref #%u: rel %u/%u/%u fork %s blk %u",
+						 block_id, rnode.spcNode, rnode.dbNode,
+						 rnode.relNode, get_forkname(forknum), blk);
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	values[7] = CStringGetTextDatum(rec_blk_ref.data);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[8] = PointerGetDatum(data);
+	values[9] = UInt32GetDatum(main_data_len);
+}
+
+/*
+ * Get the WAL record info and data.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	Assert(record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get the WAL record info and data between start LSN and end LSN.
+ */
+Datum
+pg_get_wal_record_info_2(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	if (XLogRecPtrIsInvalid(start_lsn) || XLogRecPtrIsInvalid(end_lsn))
+		PG_RETURN_NULL();
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		Assert(record);
+
+		if (xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls);
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+
+	PG_RETURN_VOID();
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+static void
+StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record)
+{
+	RmgrId		rmid;
+	uint8		recid;
+	uint32		rec_len;
+	uint32		fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics. */
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
+/*
+ * Display a single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls)
+{
+	double		n_pct;
+	double		rec_len_pct;
+	double		fpi_len_pct;
+	double		tot_len_pct;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[0] = CStringGetTextDatum(name);
+	values[1] = Int64GetDatum(n);
+	values[2] = Float4GetDatum(n_pct);
+	values[3] = Int64GetDatum(rec_len);
+	values[4] = Float4GetDatum(rec_len_pct);
+	values[5] = Int64GetDatum(fpi_len);
+	values[6] = Float4GetDatum(fpi_len_pct);
+	values[7] = Int64GetDatum(tot_len);
+	values[8] = Float4GetDatum(tot_len_pct);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+					TupleDesc tupdesc, Datum *values, bool *nulls)
+{
+	int			ri;
+	uint64		total_count = 0;
+	uint64		total_rec_len = 0;
+	uint64		total_fpi_len = 0;
+	uint64		total_len = 0;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+}
+
+/*
+ * Get the WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+					XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogRecStats stats;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_STATS_COLS];
+	bool		nulls[PG_GET_WAL_STATS_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		Assert(record);
+
+		if (xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		StoreXLogRecordStats(&stats, xlogreader);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, tupstore, tupdesc, values, nulls);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get the WAL stats between start LSN and end LSN.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	if (XLogRecPtrIsInvalid(start_lsn) || XLogRecPtrIsInvalid(end_lsn))
+		PG_RETURN_NULL();
+
+	GetWalStatsInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index f39f8044a9..8bf14a6a2a 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -814,6 +814,27 @@ ValidXLogRecord(XLogReaderState *state, XLogRecord *record, XLogRecPtr recptr)
 	return true;
 }
 
+/*
+ * Same as ValidXLogRecord but without XLogReaderState and XLogRecPtr.
+ */
+bool
+IsXLogRecordValid(XLogRecord *record)
+{
+	pg_crc32c	crc;
+
+	/* Calculate the CRC. */
+	INIT_CRC32C(crc);
+	COMP_CRC32C(crc, ((char *) record) + SizeOfXLogRecord, record->xl_tot_len - SizeOfXLogRecord);
+	/* Include the record header last. */
+	COMP_CRC32C(crc, (char *) record, offsetof(XLogRecord, xl_crc));
+	FIN_CRC32C(crc);
+
+	if (!EQ_CRC32C(record->xl_crc, crc))
+		return false;
+
+	return true;
+}
+
 /*
  * Validate a page header.
  *
@@ -956,13 +977,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1080,6 +1094,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 1f5c426ec0..41faa00868 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,12 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+const char *const
+get_forkname(ForkNumber num)
+{
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 898df2ee03..7ca690852a 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -60,7 +60,7 @@ extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 extern bool reachedConsistency;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index c0da76cab4..536741a24a 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -314,7 +314,7 @@ typedef struct RmgrData
 	void		(*rm_mask) (char *pagedata, BlockNumber blkno);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index de6fd791fe..42ddfde56e 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
@@ -288,6 +286,8 @@ extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
 extern bool XLogReaderValidatePageHeader(XLogReaderState *state,
 										 XLogRecPtr recptr, char *phdr);
 
+extern bool IsXLogRecordValid(XLogRecord *record);
+
 /*
  * Error information from WALRead that both backend and frontend caller can
  * process.  Currently only errors from pg_pread can be reported.
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a44be11ca0..27215d075c 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.33.0.windows.2

v2-0002-pg_walinspect-tests.patchapplication/octet-stream; name=v2-0002-pg_walinspect-tests.patchDownload
From a9d061ca82a49fbe8044de1925247f300149a71e Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 17 Nov 2021 13:31:31 +0000
Subject: [PATCH v2] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 37 +++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 33 +++++++++++++++++
 2 files changed, 70 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..f15ec0e5e8
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,37 @@
+CREATE EXTENSION pg_walinspect;
+SELECT pg_current_wal_lsn() AS wal_lsn \gset
+CREATE TABLE sample_tbl(col1 int, col2 int);
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+CHECKPOINT;
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn') AS valid_wal_lsn \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+SELECT record AS raw_wal_rec FROM pg_get_raw_wal_record(:'valid_wal_lsn') \gset
+SELECT * FROM pg_verify_raw_wal_record(:'raw_wal_rec');
+ is_valid 
+----------
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+CHECKPOINT;
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn2') AS valid_wal_lsn2 \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info_2(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..ac0404bb90
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,33 @@
+CREATE EXTENSION pg_walinspect;
+
+SELECT pg_current_wal_lsn() AS wal_lsn \gset
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+
+CHECKPOINT;
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn');
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn') AS valid_wal_lsn \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn');
+
+SELECT record AS raw_wal_rec FROM pg_get_raw_wal_record(:'valid_wal_lsn') \gset
+
+SELECT * FROM pg_verify_raw_wal_record(:'raw_wal_rec');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn');
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+
+CHECKPOINT;
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn2') AS valid_wal_lsn2 \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info_2(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
-- 
2.25.1

#17Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Bharath Rupireddy (#16)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Nov 25, 2021 at 3:49 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Thanks all. Here's the v1 patch set for the new extension pg_walinspect. Note that I didn't include the documentation part now, I will be doing it a bit later.

Please feel free to review and provide your thoughts.

The v1 patch set was failing to compile on Windows. Here's the v2
patch set fixing that.

I forgot to specify this: the v1 patch set was failing to compile on
Windows with errors shown at [1](Link target) -> pg_walinspect.obj : error LNK2001: unresolved external symbol forkNames [C:\Users\bhara\postgres\pg_walinspect.vcxproj] pg_walinspect.obj : error LNK2001: unresolved external symbol pg_comp_crc32c [C:\Users\bhara\postgres\pg_walinspect.vcxproj] pg_walinspect.obj : error LNK2001: unresolved external symbol wal_segment_size [C:\Users\bhara\postgres\pg_walinspect.vcxproj] pg_walinspect.obj : error LNK2001: unresolved external symbol RmgrTable [C:\Users\bhara\postgres\pg_walinspect.vcxproj] .\Release\pg_walinspect\pg_walinspect.dll : fatal error LNK1120: 4 unresolved externals [C:\Users\bhara\postgres\pg_walinspect.vcxproj]. Thanks to Julien Rouhaud who
suggested to use PGDLLIMPORT in an off-list discussion.

[1]: (Link target) -> pg_walinspect.obj : error LNK2001: unresolved external symbol forkNames [C:\Users\bhara\postgres\pg_walinspect.vcxproj] pg_walinspect.obj : error LNK2001: unresolved external symbol pg_comp_crc32c [C:\Users\bhara\postgres\pg_walinspect.vcxproj] pg_walinspect.obj : error LNK2001: unresolved external symbol wal_segment_size [C:\Users\bhara\postgres\pg_walinspect.vcxproj] pg_walinspect.obj : error LNK2001: unresolved external symbol RmgrTable [C:\Users\bhara\postgres\pg_walinspect.vcxproj] .\Release\pg_walinspect\pg_walinspect.dll : fatal error LNK1120: 4 unresolved externals [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
pg_walinspect.obj : error LNK2001: unresolved external symbol
forkNames [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
pg_walinspect.obj : error LNK2001: unresolved external symbol
pg_comp_crc32c [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
pg_walinspect.obj : error LNK2001: unresolved external symbol
wal_segment_size [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
pg_walinspect.obj : error LNK2001: unresolved external symbol
RmgrTable [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
.\Release\pg_walinspect\pg_walinspect.dll : fatal error LNK1120: 4
unresolved externals [C:\Users\bhara\postgres\pg_walinspect.vcxproj]

5 Error(s)

Regards,
Bharath Rupireddy.

#18Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Bharath Rupireddy (#17)
2 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Nov 25, 2021 at 5:54 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

On Thu, Nov 25, 2021 at 3:49 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Thanks all. Here's the v1 patch set for the new extension pg_walinspect. Note that I didn't include the documentation part now, I will be doing it a bit later.

Please feel free to review and provide your thoughts.

The v1 patch set was failing to compile on Windows. Here's the v2
patch set fixing that.

I forgot to specify this: the v1 patch set was failing to compile on
Windows with errors shown at [1]. Thanks to Julien Rouhaud who
suggested to use PGDLLIMPORT in an off-list discussion.

[1] (Link target) ->
pg_walinspect.obj : error LNK2001: unresolved external symbol
forkNames [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
pg_walinspect.obj : error LNK2001: unresolved external symbol
pg_comp_crc32c [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
pg_walinspect.obj : error LNK2001: unresolved external symbol
wal_segment_size [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
pg_walinspect.obj : error LNK2001: unresolved external symbol
RmgrTable [C:\Users\bhara\postgres\pg_walinspect.vcxproj]
.\Release\pg_walinspect\pg_walinspect.dll : fatal error LNK1120: 4
unresolved externals [C:\Users\bhara\postgres\pg_walinspect.vcxproj]

5 Error(s)

Here's the v3 patch-set with fixes for the compiler warnings reported
in the cf bot at
https://cirrus-ci.com/task/4979131497578496?logs=gcc_warning#L506.

Please review.

Regards,
Bharath Rupireddy.

Attachments:

v3-0001-pg_walinspect.patchapplication/octet-stream; name=v3-0001-pg_walinspect.patchDownload
From 6c9dcc3fa9d4ea0ab6fc82f07985465d0b4fca53 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Tue, 4 Jan 2022 16:29:33 +0000
Subject: [PATCH v3] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 106 +++
 contrib/pg_walinspect/pg_walinspect.c        | 770 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  35 +-
 src/common/relpath.c                         |   6 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   4 +-
 src/include/common/relpath.h                 |   1 +
 12 files changed, 951 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 87bf87ab90..780059ce66 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -39,6 +39,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..bd641abd04
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,106 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+--
+-- pg_get_first_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_valid_wal_record_lsn(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_valid_wal_record_lsn'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) TO pg_monitor;
+
+--
+-- pg_verify_raw_wal_record()
+--
+CREATE FUNCTION pg_verify_raw_wal_record(IN record bytea,
+    OUT is_valid boolean
+)
+AS 'MODULE_PATHNAME', 'pg_verify_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_verify_raw_wal_record(bytea) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_verify_raw_wal_record(bytea) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT rmgr text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info_2()
+--
+CREATE FUNCTION pg_get_wal_record_info_2(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT rmgr text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info_2'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info_2(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info_2(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT rmgr text,
+    OUT count int8,
+    OUT count_per float4,
+    OUT record_size int8,
+    OUT record_size_per float4,
+    OUT fpi_size int8,
+    OUT fpi_size_per float4,
+    OUT combined_size int8,
+    OUT combined_size_per float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..80651e1cbc
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,770 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Portions Copyright (c) 2012-2021, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+PG_MODULE_MAGIC;
+
+typedef struct Stats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+} Stats;
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	Stats		rmgr_stats[RM_NEXT_ID];
+	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+}			XLogRecStats;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info_2);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record,
+											bool warning);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls);
+static void StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record);
+static void GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+								TupleDesc tupdesc, Datum *values, bool *nulls);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len, uint64 fpi_len,
+							 uint64 total_fpi_len, uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls);
+static void GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+								XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Intialize WAL reader and identify the first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record, bool warning)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (!xlogreader)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	/*
+	 * Display a message that we're skipping data if the given lsn wasn't a
+	 * pointer to the start of a record and also wasn't a pointer to the
+	 * beginning of a segment (e.g. we were used in file mode).
+	 */
+	if (warning && *first_record != lsn &&
+		XLogSegmentOffset(lsn, wal_segment_size) != 0)
+		ereport(WARNING,
+				(errmsg_plural("first record is after %X/%X, at %X/%X, skipping over %u byte",
+							   "first record is after %X/%X, at %X/%X, skipping over %u bytes",
+							   (*first_record - lsn),
+							   LSN_FORMAT_ARGS(lsn),
+							   LSN_FORMAT_ARGS(*first_record),
+							   (uint32) (*first_record - lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read the WAL next record
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	char	   *errormsg;
+	XLogRecord *record;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (!record)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get the raw WAL record. This function will wait for the required WAL at LSN.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	   *raw_record;
+	uint32		rec_len;
+	char	   *raw_record_data;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	Assert(record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	/*
+	 * XXX: Check the authenticity of the WAL record here? Although it might
+	 * have been done by XLogReadRecord, after memcpy, just check for
+	 * raw_record_data correctness.
+	 */
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[0] = LSNGetDatum(first_record);
+	values[1] = PointerGetDatum(raw_record);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_COLS
+}
+
+/*
+ * Get the raw WAL record. This function will wait for the required WAL at LSN.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	XLogReaderFree(xlogreader);
+
+	PG_RETURN_LSN(first_record);
+}
+
+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{
+	bytea	   *raw_record;
+	XLogRecord *record;
+	bool		valid;
+
+	raw_record = PG_GETARG_BYTEA_PP(0);
+	record = (XLogRecord *) VARDATA_ANY(raw_record);
+	valid = IsXLogRecordValid(record);
+
+	PG_RETURN_BOOL(valid);
+}
+
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+static void
+GetXLogRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
+{
+	int			block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data. It doesn't seem worth it to
+	 * add an accessor macro for this.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += record->blocks[block_id].bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Get the WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32		rec_len;
+	uint32		fpi_len;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int			block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	   *data;
+	char	   *main_data;
+	uint32		main_data_len;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	values[0] = LSNGetDatum(lsn);
+	values[1] = LSNGetDatum(XLogRecGetPrev(record));
+	values[2] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[3] = CStringGetTextDatum(desc->rm_name);
+	values[4] = UInt32GetDatum(rec_len);
+	values[5] = UInt32GetDatum(XLogRecGetTotalLen(record));
+
+	initStringInfo(&rec_desc);
+
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "desc: UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "desc: %s ", id);
+
+	initStringInfo(&temp);
+
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+
+	values[6] = CStringGetTextDatum(rec_desc.data);
+
+	pfree(temp.data);
+
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+		appendStringInfo(&rec_blk_ref, "blkref #%u: rel %u/%u/%u fork %s blk %u",
+						 block_id, rnode.spcNode, rnode.dbNode,
+						 rnode.relNode, get_forkname(forknum), blk);
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	values[7] = CStringGetTextDatum(rec_blk_ref.data);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[8] = PointerGetDatum(data);
+	values[9] = UInt32GetDatum(main_data_len);
+}
+
+/*
+ * Get the WAL record info and data.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (XLogRecPtrIsInvalid(lsn))
+		PG_RETURN_NULL();
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get the WAL record info and data between start LSN and end LSN.
+ */
+Datum
+pg_get_wal_record_info_2(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	if (XLogRecPtrIsInvalid(start_lsn) || XLogRecPtrIsInvalid(end_lsn))
+		PG_RETURN_NULL();
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		if (xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls);
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+
+	PG_RETURN_VOID();
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+static void
+StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record)
+{
+	RmgrId		rmid;
+	uint8		recid;
+	uint32		rec_len;
+	uint32		fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics. */
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
+/*
+ * Display a single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls)
+{
+	double		n_pct;
+	double		rec_len_pct;
+	double		fpi_len_pct;
+	double		tot_len_pct;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[0] = CStringGetTextDatum(name);
+	values[1] = Int64GetDatum(n);
+	values[2] = Float4GetDatum(n_pct);
+	values[3] = Int64GetDatum(rec_len);
+	values[4] = Float4GetDatum(rec_len_pct);
+	values[5] = Int64GetDatum(fpi_len);
+	values[6] = Float4GetDatum(fpi_len_pct);
+	values[7] = Int64GetDatum(tot_len);
+	values[8] = Float4GetDatum(tot_len_pct);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+					TupleDesc tupdesc, Datum *values, bool *nulls)
+{
+	int			ri;
+	uint64		total_count = 0;
+	uint64		total_rec_len = 0;
+	uint64		total_fpi_len = 0;
+	uint64		total_len = 0;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+}
+
+/*
+ * Get the WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+					XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogRecStats stats;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_STATS_COLS];
+	bool		nulls[PG_GET_WAL_STATS_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		if (xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		StoreXLogRecordStats(&stats, xlogreader);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, tupstore, tupdesc, values, nulls);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get the WAL stats between start LSN and end LSN.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	if (XLogRecPtrIsInvalid(start_lsn) || XLogRecPtrIsInvalid(end_lsn))
+		PG_RETURN_NULL();
+
+	GetWalStatsInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index 3a7de02565..22908e2e3d 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -814,6 +814,27 @@ ValidXLogRecord(XLogReaderState *state, XLogRecord *record, XLogRecPtr recptr)
 	return true;
 }
 
+/*
+ * Same as ValidXLogRecord but without XLogReaderState and XLogRecPtr.
+ */
+bool
+IsXLogRecordValid(XLogRecord *record)
+{
+	pg_crc32c	crc;
+
+	/* Calculate the CRC. */
+	INIT_CRC32C(crc);
+	COMP_CRC32C(crc, ((char *) record) + SizeOfXLogRecord, record->xl_tot_len - SizeOfXLogRecord);
+	/* Include the record header last. */
+	COMP_CRC32C(crc, (char *) record, offsetof(XLogRecord, xl_crc));
+	FIN_CRC32C(crc);
+
+	if (!EQ_CRC32C(record->xl_crc, crc))
+		return false;
+
+	return true;
+}
+
 /*
  * Validate a page header.
  *
@@ -956,13 +977,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1080,6 +1094,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 1f5c426ec0..41faa00868 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,12 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+const char *const
+get_forkname(ForkNumber num)
+{
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 34f6c89f06..5384c902e3 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -60,7 +60,7 @@ extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 extern bool reachedConsistency;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index c0da76cab4..536741a24a 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -314,7 +314,7 @@ typedef struct RmgrData
 	void		(*rm_mask) (char *pagedata, BlockNumber blkno);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index de6fd791fe..42ddfde56e 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
@@ -288,6 +286,8 @@ extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
 extern bool XLogReaderValidatePageHeader(XLogReaderState *state,
 										 XLogRecPtr recptr, char *phdr);
 
+extern bool IsXLogRecordValid(XLogRecord *record);
+
 /*
  * Error information from WALRead that both backend and frontend caller can
  * process.  Currently only errors from pg_pread can be reported.
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a44be11ca0..27215d075c 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v3-0001-pg_walinspect-tests.patchapplication/octet-stream; name=v3-0001-pg_walinspect-tests.patchDownload
From b67fa123dc5d5bbf1dc6f5acca91cfd64147c222 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Tue, 4 Jan 2022 16:28:39 +0000
Subject: [PATCH v3] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 37 +++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 33 +++++++++++++++++
 2 files changed, 70 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..f15ec0e5e8
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,37 @@
+CREATE EXTENSION pg_walinspect;
+SELECT pg_current_wal_lsn() AS wal_lsn \gset
+CREATE TABLE sample_tbl(col1 int, col2 int);
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+CHECKPOINT;
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn') AS valid_wal_lsn \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+SELECT record AS raw_wal_rec FROM pg_get_raw_wal_record(:'valid_wal_lsn') \gset
+SELECT * FROM pg_verify_raw_wal_record(:'raw_wal_rec');
+ is_valid 
+----------
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn');
+ ok 
+----
+ t
+(1 row)
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+CHECKPOINT;
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn2') AS valid_wal_lsn2 \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info_2(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..ac0404bb90
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,33 @@
+CREATE EXTENSION pg_walinspect;
+
+SELECT pg_current_wal_lsn() AS wal_lsn \gset
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+
+CHECKPOINT;
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn');
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn') AS valid_wal_lsn \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn');
+
+SELECT record AS raw_wal_rec FROM pg_get_raw_wal_record(:'valid_wal_lsn') \gset
+
+SELECT * FROM pg_verify_raw_wal_record(:'raw_wal_rec');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn');
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+
+CHECKPOINT;
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn2') AS valid_wal_lsn2 \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info_2(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn', :'valid_wal_lsn2') \gset
-- 
2.25.1

#19Greg Stark
stark@mit.edu
In reply to: Bharath Rupireddy (#18)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

So I looked at this patch and I have the same basic question as Bruce.
Do we really want to expose every binary tool associated with Postgres
as an extension? Obviously this is tempting for cloud provider users
which is not an unreasonable argument. But it does have consequences.

1) Some things like pg_waldump are running code that is not normally
under user control. This could have security issues or reliability
issues.

On that front I'm especially concerned that pg_verify_raw_wal_record()
for example would let an attacker feed arbitrary hand crafted xlog
records into the parser which is not normally something a user can do.
If they feed it something it's not expecting it might be easy to cause
a crash and server restart.

There's also a bit of concern about data retention. Generally in
Postgres when rows are deleted there's very weak guarantees about the
data really being wiped. We definitely don't wipe it from disk of
course. And things like pageinspect could expose it long after it's
been deleted. But one might imagine after pageinspect shows it's gone
and/or after a vacuum full the data is actually purged. But then
something like pg_walinspect would make even that insufficient.

2) There's no documentation. I'm guessing you hesitated to write
documentation until the interface is settled but actually sometimes
writing documentation helps expose things in the interface that look
strange when you try to explain them.

3) And the interface does look a bit strange. Like what's the deal
with pg_get_wal_record_info_2() ? I gather it's just a SRF version of
pg_get_wal_record_info() but that's a strange name. And then what's
the point of pg_get_wal_record_info() at all? Why wouldn't the SRF be
sufficient even for the specific case of a single record?

And the stats functions seem a bit out of place to me. If the SRF
returned the data in the right format the user should be able to do
aggregate queries to generate these stats easily enough. If anything a
simple SQL function to do the aggregations could be provided.

Now this is starting to get into the realm of bikeshedding but... Some
of the code is taken straight from pg_waldump and does things like:

+ appendStringInfo(&rec_blk_ref, "blkref #%u: rel %u/%u/%u fork %s blk %u",

But that's kind of out of place for an SQL interface. It makes it hard
to write queries since things like the relid, block number etc are in
the string. If I wanted to use these functions I would expect to be
doing something like "select all the decoded records pertaining to
block n".

All that said, I don't want to gatekeep based on this kind of
criticism. The existing code is based on pg_waldump and if we want an
extension to expose that then that's a reasonable place to start. We
can work on a better format for the data later it doesn't mean we
shouldn't start with something we have today.

4) This isn't really an issue with your patch at all but why on earth
do we have a bitvector for WAL compression methods?! Like, what does
it mean to have multiple compression methods set? That should just be
a separate field with values for each type of compression surely?

I suppose this raises the issue of what happens if someone fixes that.
They'll now have to update pg_waldump *and* pg_walinspect. I don't
think that would actually be a lot of work but it's definitely more
than just one. Also, perhaps they should be in the same contrib
directory so at least people won't forget there are two.

#20Justin Pryzby
pryzby@telsasoft.com
In reply to: Greg Stark (#19)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Mon, Jan 31, 2022 at 04:40:09PM -0500, Greg Stark wrote:

4) This isn't really an issue with your patch at all but why on earth
do we have a bitvector for WAL compression methods?! Like, what does
it mean to have multiple compression methods set? That should just be
a separate field with values for each type of compression surely?

I don't have an answer to your question, but the discussion was here.

In the versions of the patches I sent on Mar 15, Mar 21, May 18, May 24, Jun
13, I avoided "one bit per compression method", but Michael thought this was
simpler.

/messages/by-id/20210622031358.GF29179@telsasoft.com
On Mon, Jun 21, 2021 at 10:13:58PM -0500, Justin Pryzby wrote:

+/* compression methods supported */
+#define BKPIMAGE_COMPRESS_PGLZ 0x04
+#define BKPIMAGE_COMPRESS_ZLIB 0x08
+#define BKPIMAGE_COMPRESS_LZ4  0x10
+#define BKPIMAGE_COMPRESS_ZSTD 0x20
+#define        BKPIMAGE_IS_COMPRESSED(info) \
+       ((info & (BKPIMAGE_COMPRESS_PGLZ | BKPIMAGE_COMPRESS_ZLIB | \
+                         BKPIMAGE_COMPRESS_LZ4 | BKPIMAGE_COMPRESS_ZSTD)) != 0)

You encouraged saving bits here, so I'm surprised to see that your patches
use one bit per compression method: 2 bits to support no/pglz/lz4, 3 to add
zstd, and the previous patch used 4 bits to also support zlib.

There are spare bits available for that, but now there can be an inconsistency
if two bits are set. Also, 2 bits could support 4 methods (including "no").

On Tue, Jun 22, 2021 at 12:53:46PM +0900, Michael Paquier wrote:

Yeah, I know. I have just finished with that to get something
readable for the sake of the tests. As you say, the point is moot
just we add one new method, anyway, as we need just one new bit.
And that's what I would like to do for v15 with LZ4 as the resulting
patch is simple. It would be an idea to discuss more compression
methods here once we hear more from users when this is released in the
field, re-considering at this point if more is necessary or not.

#21Greg Stark
stark@mit.edu
In reply to: Greg Stark (#19)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Additionally I've looked at the tests and I'm not sure but I don't
think this arrangement is going to work. I don't have the time to run
CLOBBER_CACHE and CLOBBER_CACHE_ALWAYS tests but I know they run
*really* slowly. So the test can't just do a CHECKPOINT and then trust
that the next few transactions will still be in the wal to decode
later. There could have been many more timed checkpoints in between.

I think the way to do it is to create either a backup label or a
replication slot. Then you can inspect the lsn of the label or slot
and decode all transactions between that point and the current point.

I also think you should try to have a wider set of wal records in that
range to test decoding records with and without full page writes, with
DDL records, etc.

I do like that the tests don't actually have the decoded record info
in the test though. But they can do a minimal effort to check that the
records they think they're testing are actually being tested. Insert
into a temporary table and then run a few queries with WHERE clauses
to test for a heap insert, btree insert test the right relid is
present, and test that a full page write is present (if full page
writes are enabled I guess). You don't need an exhaustive set of
checks because you're not testing that wal logging works properly,
just that the tests aren't accidentally passing because they're not
finding any interesting records.

#22Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Greg Stark (#19)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Tue, Feb 1, 2022 at 3:10 AM Greg Stark <stark@mit.edu> wrote:

So I looked at this patch and I have the same basic question as Bruce.

Thanks a lot for the comments.

Do we really want to expose every binary tool associated with Postgres
as an extension? Obviously this is tempting for cloud provider users
which is not an unreasonable argument. But it does have consequences.

Perhaps not every tool needs to be exposed, but given the advantages
that pg_walinspect can provide it's a good candidate to have it as a
core extension. Some of the advantages are - debugging, WAL analysis,
feeding WAL stats and info to dashboards to show customers and answer
their queries, RCA etc., for educational purposes - one can understand
the WAL structure, stats, different record types etc. Another nice
thing is getting raw WAL data out of the running server (of course all
the users can't get it only the allowed ones, currently users with
pg_monitor role, if required we can change it to some other predefined
role). For instance, the raw WAL data can be fed to external page
repair tools to apply on a raw page (one can get this from pageinspect
extension).

1) Some things like pg_waldump are running code that is not normally
under user control. This could have security issues or reliability
issues.

I understand this and also I think the same concern applies to
pageinspect tool which exposes getting raw page data. The
pg_walinspect functions are currently accessible by the users with
pg_monitor role, if required we can change this to some other
predefined role.

On that front I'm especially concerned that pg_verify_raw_wal_record()
for example would let an attacker feed arbitrary hand crafted xlog
records into the parser which is not normally something a user can do.
If they feed it something it's not expecting it might be easy to cause
a crash and server restart.

This function does nothing (no writes) to the server but just checking
the CRC of the WAL record. If at all one can make the server crash
with an input, then that should be a problem with the server code
which needs to be fixed. But IMO this function doesn't have a concern
as such.

There's also a bit of concern about data retention. Generally in
Postgres when rows are deleted there's very weak guarantees about the
data really being wiped. We definitely don't wipe it from disk of
course. And things like pageinspect could expose it long after it's
been deleted. But one might imagine after pageinspect shows it's gone
and/or after a vacuum full the data is actually purged. But then
something like pg_walinspect would make even that insufficient.

The idea of pg_walinspect is to get the WAL info, data and stats out
of a running postgres server, if the WAL isn't available, the
functions would say that.

2) There's no documentation. I'm guessing you hesitated to write
documentation until the interface is settled but actually sometimes
writing documentation helps expose things in the interface that look
strange when you try to explain them.

I will send out the new patch set with documentation soon.

3) And the interface does look a bit strange. Like what's the deal
with pg_get_wal_record_info_2() ? I gather it's just a SRF version of
pg_get_wal_record_info() but that's a strange name. And then what's
the point of pg_get_wal_record_info() at all? Why wouldn't the SRF be
sufficient even for the specific case of a single record?

I agree, pg_get_wal_record_info_2 is a poor naming.
pg_get_wal_record_info_2 takes range of LSN (start and end) to give
the wal info, whereas pg_get_wal_record_info just takes one LSN. Maybe
I will change pg_get_wal_record_info_2 to pg_get_wal_record_info_range
or pg_get_wal_records_info or someother namign is better? If the
suggestion is to overload pg_get_wal_record_info one with single LSN
and another with start and end LSNs, I'm okay with that too.
Otherwise, I can have pg_get_wal_record_info with start and end LSN
(end LSN default to NULL) and return setof record.

And the stats functions seem a bit out of place to me. If the SRF
returned the data in the right format the user should be able to do
aggregate queries to generate these stats easily enough. If anything a
simple SQL function to do the aggregations could be provided.

Now this is starting to get into the realm of bikeshedding but... Some
of the code is taken straight from pg_waldump and does things like:

+ appendStringInfo(&rec_blk_ref, "blkref #%u: rel %u/%u/%u fork %s blk %u",

But that's kind of out of place for an SQL interface. It makes it hard
to write queries since things like the relid, block number etc are in
the string. If I wanted to use these functions I would expect to be
doing something like "select all the decoded records pertaining to
block n".

I will think more about this and change it in the next version of the
patch set, perhaps I will add more columns to the functions.

All that said, I don't want to gatekeep based on this kind of
criticism. The existing code is based on pg_waldump and if we want an
extension to expose that then that's a reasonable place to start. We
can work on a better format for the data later it doesn't mean we
shouldn't start with something we have today.

IMO, we can always extend the functions in future, once the
pg_walinspect extension gets in with minimum number of much-required
and basic functions.

I suppose this raises the issue of what happens if someone fixes that.
They'll now have to update pg_waldump *and* pg_walinspect. I don't
think that would actually be a lot of work but it's definitely more
than just one. Also, perhaps they should be in the same contrib
directory so at least people won't forget there are two.

Currently, all the tools are placed in src/bin and extensions are in
contrib directory. I don't think we ever keep the extension in src/bin
or vice versa. Having said, this maybe we can add comments on having
to change/fix in both pg_waldump and pg_walinspect. We also have to
deal with this situation in some of the existing tools such as
pg_controldata.

Regards,
Bharath Rupireddy.

#23Robert Haas
robertmhaas@gmail.com
In reply to: Greg Stark (#19)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Mon, Jan 31, 2022 at 4:40 PM Greg Stark <stark@mit.edu> wrote:

So I looked at this patch and I have the same basic question as Bruce.
Do we really want to expose every binary tool associated with Postgres
as an extension? Obviously this is tempting for cloud provider users
which is not an unreasonable argument. But it does have consequences.

1) Some things like pg_waldump are running code that is not normally
under user control. This could have security issues or reliability
issues.

For what it's worth, I am generally in favor of having something like
this in PostgreSQL. I think it's wrong of us to continue assuming that
everyone has command-line access. Even when that's true, it's not
necessarily convenient. If you choose to use a relational database,
you may be the sort of person who likes SQL. And if you are, you may
want to have the database tell you what's going on via SQL rather than
command-line tools or operating system utilities. Imagine if we didn't
have pg_stat_activity and you had to get that information by running a
separate binary. Would anyone like that? Why is this case any
different?

A few years ago we exposed data from pg_control via SQL and similar
concerns were raised - but it turns out to be pretty useful. I don't
know why this shouldn't be equally useful. Sure, there's some
duplication in functionality, but it's not a huge maintenance burden
for the project, and people (including me) like having it available. I
think the same things will be true here.

If decoding WAL causes security problems, that's something we better
fix, because WAL is constantly decoded on standbys and via logical
decoding on systems all over the place. I agree that we can't let
users supply their own hand-crafted WAL records to be decoded without
causing more trouble than we can handle, but if it's not safe to
decode the WAL the system generated than we are in a lot of trouble
already.

I hasten to say that I'm not endorsing every detail or indeed any
detail of the proposed patch, and some of the concerns you mention
later sound well-founded to me. But I disagree with the idea that we
shouldn't have both a command-line utility that roots through files on
disk and an SQL interface that works with a running system.

--
Robert Haas
EDB: http://www.enterprisedb.com

#24Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Robert Haas (#23)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Sun, Feb 6, 2022 at 9:15 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Jan 31, 2022 at 4:40 PM Greg Stark <stark@mit.edu> wrote:

So I looked at this patch and I have the same basic question as Bruce.
Do we really want to expose every binary tool associated with Postgres
as an extension? Obviously this is tempting for cloud provider users
which is not an unreasonable argument. But it does have consequences.

1) Some things like pg_waldump are running code that is not normally
under user control. This could have security issues or reliability
issues.

For what it's worth, I am generally in favor of having something like
this in PostgreSQL. I think it's wrong of us to continue assuming that
everyone has command-line access. Even when that's true, it's not
necessarily convenient. If you choose to use a relational database,
you may be the sort of person who likes SQL. And if you are, you may
want to have the database tell you what's going on via SQL rather than
command-line tools or operating system utilities. Imagine if we didn't
have pg_stat_activity and you had to get that information by running a
separate binary. Would anyone like that? Why is this case any
different?

A few years ago we exposed data from pg_control via SQL and similar
concerns were raised - but it turns out to be pretty useful. I don't
know why this shouldn't be equally useful. Sure, there's some
duplication in functionality, but it's not a huge maintenance burden
for the project, and people (including me) like having it available. I
think the same things will be true here.

If decoding WAL causes security problems, that's something we better
fix, because WAL is constantly decoded on standbys and via logical
decoding on systems all over the place. I agree that we can't let
users supply their own hand-crafted WAL records to be decoded without
causing more trouble than we can handle, but if it's not safe to
decode the WAL the system generated than we are in a lot of trouble
already.

I hasten to say that I'm not endorsing every detail or indeed any
detail of the proposed patch, and some of the concerns you mention
later sound well-founded to me. But I disagree with the idea that we
shouldn't have both a command-line utility that roots through files on
disk and an SQL interface that works with a running system.

Thanks Robert for your comments.

+ appendStringInfo(&rec_blk_ref, "blkref #%u: rel %u/%u/%u fork %s blk %u",

But that's kind of out of place for an SQL interface. It makes it hard
to write queries since things like the relid, block number etc are in
the string. If I wanted to use these functions I would expect to be
doing something like "select all the decoded records pertaining to
block n".

Thanks Greg for your review of the patches. Since there can be
multiple blkref for WAL record type HEAP2 (for multi inserts
basically) [1]rmgr: Heap2 len (rec/tot): 64/ 8256, tx: 0, lsn: 0/014A9070, prev 0/014A8FF8, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0 FPW, blkref #1: rel 1663/12757/16384 blk 0 rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn: 0/014AB0C8, prev 0/014A9070, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel 1663/12757/16384 blk 1 rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn: 0/014AB108, prev 0/014AB0C8, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel 1663/12757/16384 blk 2 rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn: 0/014AB148, prev 0/014AB108, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel 1663/12757/16384 blk 3 rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn: 0/014AB188, prev 0/014AB148, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel 1663/12757/16384 blk 4, I couldn't find a better way to break it and represent
it as a non-text column. IMO this is simpler and users can easily find
out answers to "how many WAL records my relation generated between
lsn1 and lsn2 or how many WAL records of type Heap exist and so on?",
see [2]postgres=# select count(*) from pg_get_wal_records_info('0/13C0A98', '0/0157A160') where block_ref like '%16384%' and rmgr like 'Heap'; count ------- 10100 (1 row). I've also added a test case to just show this in 0002 patch.

Here's the v4 patch set that has the following changes along with
Greg's review comments addressed:

1) Added documentation as 0003 patch.
2) Removed CHECKPOINT commands from tests as it is unnecessary.
3) Added input validation code and tests.
4) A few more comments have been added.
5) Currently, only superusers can create the extension, but users with
the pg_monitor role can use the functions.
6) Test cases are basic yet they cover all the functions, error cases
with input validations, I don't think we need to add many more test
cases as suggested upthread, but I'm open to add a few more if I miss
any use-case.

Please review the v4 patch set further and let me know your thoughts.

[1]: rmgr: Heap2 len (rec/tot): 64/ 8256, tx: 0, lsn: 0/014A9070, prev 0/014A8FF8, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0 FPW, blkref #1: rel 1663/12757/16384 blk 0 rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn: 0/014AB0C8, prev 0/014A9070, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel 1663/12757/16384 blk 1 rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn: 0/014AB108, prev 0/014AB0C8, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel 1663/12757/16384 blk 2 rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn: 0/014AB148, prev 0/014AB108, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel 1663/12757/16384 blk 3 rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn: 0/014AB188, prev 0/014AB148, desc: VISIBLE cutoff xid 709 flags 0x01, blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel 1663/12757/16384 blk 4
rmgr: Heap2 len (rec/tot): 64/ 8256, tx: 0, lsn:
0/014A9070, prev 0/014A8FF8, desc: VISIBLE cutoff xid 709 flags 0x01,
blkref #0: rel 1663/12757/16384 fork vm blk 0 FPW, blkref #1: rel
1663/12757/16384 blk 0
rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn:
0/014AB0C8, prev 0/014A9070, desc: VISIBLE cutoff xid 709 flags 0x01,
blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel
1663/12757/16384 blk 1
rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn:
0/014AB108, prev 0/014AB0C8, desc: VISIBLE cutoff xid 709 flags 0x01,
blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel
1663/12757/16384 blk 2
rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn:
0/014AB148, prev 0/014AB108, desc: VISIBLE cutoff xid 709 flags 0x01,
blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel
1663/12757/16384 blk 3
rmgr: Heap2 len (rec/tot): 59/ 59, tx: 0, lsn:
0/014AB188, prev 0/014AB148, desc: VISIBLE cutoff xid 709 flags 0x01,
blkref #0: rel 1663/12757/16384 fork vm blk 0, blkref #1: rel
1663/12757/16384 blk 4

[2]: postgres=# select count(*) from pg_get_wal_records_info('0/13C0A98', '0/0157A160') where block_ref like '%16384%' and rmgr like 'Heap'; count ------- 10100 (1 row)
postgres=# select count(*) from pg_get_wal_records_info('0/13C0A98',
'0/0157A160') where block_ref like '%16384%' and rmgr like 'Heap';
count
-------
10100
(1 row)

postgres=# select count(*) from t1;
count
-------
10100
(1 row)

postgres=#

postgres=# select count(*) from pg_get_wal_records_info('0/13C0A98',
'0/0157A160') where block_ref like '%FPW%';
count
-------
78
(1 row)

postgres=#

Regards,
Bharath Rupireddy.

Attachments:

v4-0001-pg_walinspect.patchapplication/octet-stream; name=v4-0001-pg_walinspect.patchDownload
From 360fa1074403181dd05b542851990f3057083ced Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 9 Feb 2022 17:59:01 +0000
Subject: [PATCH v4] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 101 +++
 contrib/pg_walinspect/pg_walinspect.c        | 831 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  28 +-
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/common/relpath.c                         |  18 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   5 +-
 src/include/common/relpath.h                 |   1 +
 13 files changed, 1012 insertions(+), 17 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index e3e221308b..705c6fc36b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -40,6 +40,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..ffd236cc00
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,101 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_first_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_valid_wal_record_lsn(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_valid_wal_record_lsn'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) TO pg_monitor;
+
+--
+-- pg_verify_raw_wal_record()
+--
+CREATE FUNCTION pg_verify_raw_wal_record(IN record bytea,
+    OUT is_valid boolean
+)
+AS 'MODULE_PATHNAME', 'pg_verify_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_verify_raw_wal_record(bytea) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..6f53ee4c6b
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,831 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+typedef struct Stats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+} Stats;
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	Stats		rmgr_stats[RM_NEXT_ID];
+	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+}	XLogRecStats;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record,
+											bool warning);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls);
+static void StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record);
+static void GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+								TupleDesc tupdesc, Datum *values, bool *nulls);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len, uint64 fpi_len,
+							 uint64 total_fpi_len, uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls);
+static void GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+								XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Intialize WAL reader and identify the first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record, bool warning)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (!xlogreader)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	/*
+	 * Display a message that we're skipping data if the given lsn wasn't a
+	 * pointer to the start of a record and also wasn't a pointer to the
+	 * beginning of a segment (e.g. we were used in file mode).
+	 */
+	if (warning && *first_record != lsn &&
+		XLogSegmentOffset(lsn, wal_segment_size) != 0)
+		ereport(WARNING,
+				(errmsg_plural("first record is after %X/%X, at %X/%X, skipping over %u byte",
+							   "first record is after %X/%X, at %X/%X, skipping over %u bytes",
+							   (*first_record - lsn),
+							   LSN_FORMAT_ARGS(lsn),
+							   LSN_FORMAT_ARGS(*first_record),
+							   (uint32) (*first_record - lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read the WAL next record
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	char	   *errormsg;
+	XLogRecord *record;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (!record)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get the raw WAL record.
+ *
+ * XXX: Currently, this function will wait if the future WAL LSN is provided.
+ * May be this could be improved by checking if the given LSN is less than or
+ * equal to the current flush LSN using GetFlushRecPtr.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	   *raw_record;
+	uint32		rec_len;
+	char	   *raw_record_data;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	Assert(record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[0] = LSNGetDatum(first_record);
+	values[1] = PointerGetDatum(raw_record);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_COLS
+}
+
+/*
+ * Get the first valid raw WAL record lsn.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	XLogReaderFree(xlogreader);
+
+	PG_RETURN_LSN(first_record);
+}
+
+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{
+	bytea	   *raw_record;
+	XLogRecord *record;
+	bool		valid;
+
+	raw_record = PG_GETARG_BYTEA_PP(0);
+	record = (XLogRecord *) VARDATA_ANY(raw_record);
+	valid = ValidXLogRecord(NULL, record, InvalidXLogRecPtr);
+
+	PG_RETURN_BOOL(valid);
+}
+
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+static void
+GetXLogRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
+{
+	int			block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data. It doesn't seem worth it to
+	 * add an accessor macro for this.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += record->blocks[block_id].bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Get the WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32		rec_len;
+	uint32		fpi_len;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int			block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	   *data;
+	char	   *main_data;
+	uint32		main_data_len;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	values[0] = LSNGetDatum(lsn);
+	values[1] = LSNGetDatum(XLogRecGetPrev(record));
+	values[2] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[3] = CStringGetTextDatum(desc->rm_name);
+	values[4] = UInt32GetDatum(rec_len);
+	values[5] = UInt32GetDatum(XLogRecGetTotalLen(record));
+
+	initStringInfo(&rec_desc);
+
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+
+	values[6] = CStringGetTextDatum(rec_desc.data);
+
+	pfree(temp.data);
+
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		}
+		else
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+		}
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	values[7] = CStringGetTextDatum(rec_blk_ref.data);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[8] = PointerGetDatum(data);
+	values[9] = UInt32GetDatum(main_data_len);
+}
+
+/*
+ * Get the WAL record info and data.
+ *
+ * XXX: Currently, this function will wait if the future WAL LSN is provided.
+ * May be this could be improved by checking if the given LSN is less than or
+ * equal to the current flush LSN using GetFlushRecPtr.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get all the WAL records info and data between start LSN and end LSN.
+ *
+ * XXX: Currently, this function will wait if the future WAL LSN is provided.
+ * May be this could be improved by checking if the given LSN is less than or
+ * equal to the current flush LSN using GetFlushRecPtr.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	/* Validate input */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	if (XLogRecPtrIsInvalid(end_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record end LSN")));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL record start LSN must be less than end LSN")));
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		if (xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls);
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+
+	PG_RETURN_VOID();
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+static void
+StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record)
+{
+	RmgrId		rmid;
+	uint8		recid;
+	uint32		rec_len;
+	uint32		fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics. */
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
+/*
+ * Display a single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls)
+{
+	double		n_pct;
+	double		rec_len_pct;
+	double		fpi_len_pct;
+	double		tot_len_pct;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[0] = CStringGetTextDatum(name);
+	values[1] = Int64GetDatum(n);
+	values[2] = Float4GetDatum(n_pct);
+	values[3] = Int64GetDatum(rec_len);
+	values[4] = Float4GetDatum(rec_len_pct);
+	values[5] = Int64GetDatum(fpi_len);
+	values[6] = Float4GetDatum(fpi_len_pct);
+	values[7] = Int64GetDatum(tot_len);
+	values[8] = Float4GetDatum(tot_len_pct);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+					TupleDesc tupdesc, Datum *values, bool *nulls)
+{
+	int			ri;
+	uint64		total_count = 0;
+	uint64		total_rec_len = 0;
+	uint64		total_fpi_len = 0;
+	uint64		total_len = 0;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+}
+
+/*
+ * Get the WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+					XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogRecStats stats;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_STATS_COLS];
+	bool		nulls[PG_GET_WAL_STATS_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		if (xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		StoreXLogRecordStats(&stats, xlogreader);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, tupstore, tupdesc, values, nulls);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get the WAL stats between start LSN and end LSN.
+ *
+ * XXX: Currently, this function will wait if the future WAL LSN is provided.
+ * May be this could be improved by checking if the given LSN is less than or
+ * equal to the current flush LSN using GetFlushRecPtr.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	/* Validate input */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	if (XLogRecPtrIsInvalid(end_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record end LSN")));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL record start LSN must be less than end LSN")));
+
+	GetWalStatsInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index 35029cf97d..13e3e8574b 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -44,8 +44,6 @@ static int	ReadPageInternal(XLogReaderState *state, XLogRecPtr pageptr,
 static void XLogReaderInvalReadState(XLogReaderState *state);
 static bool ValidXLogRecordHeader(XLogReaderState *state, XLogRecPtr RecPtr,
 								  XLogRecPtr PrevRecPtr, XLogRecord *record, bool randAccess);
-static bool ValidXLogRecord(XLogReaderState *state, XLogRecord *record,
-							XLogRecPtr recptr);
 static void ResetDecoder(XLogReaderState *state);
 static void WALOpenSegmentInit(WALOpenSegment *seg, WALSegmentContext *segcxt,
 							   int segsize, const char *waldir);
@@ -791,7 +789,7 @@ ValidXLogRecordHeader(XLogReaderState *state, XLogRecPtr RecPtr,
  * record's header, which means in particular that xl_tot_len is at least
  * SizeOfXLogRecord.
  */
-static bool
+bool
 ValidXLogRecord(XLogReaderState *state, XLogRecord *record, XLogRecPtr recptr)
 {
 	pg_crc32c	crc;
@@ -805,9 +803,13 @@ ValidXLogRecord(XLogReaderState *state, XLogRecord *record, XLogRecPtr recptr)
 
 	if (!EQ_CRC32C(record->xl_crc, crc))
 	{
-		report_invalid_record(state,
-							  "incorrect resource manager data checksum in record at %X/%X",
-							  LSN_FORMAT_ARGS(recptr));
+		if (state && !XLogRecPtrIsInvalid(recptr))
+		{
+			report_invalid_record(state,
+								  "incorrect resource manager data checksum in record at %X/%X",
+								  LSN_FORMAT_ARGS(recptr));
+		}
+
 		return false;
 	}
 
@@ -956,13 +958,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1080,6 +1075,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index a6251e1a96..f8d61b56ab 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index a4b1c1286f..cc1b867f58 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -59,7 +59,7 @@ extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 extern bool reachedConsistency;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 849954a8e5..38fd51bafa 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 477f0efe26..304f7f56ce 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
@@ -288,6 +286,9 @@ extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
 extern bool XLogReaderValidatePageHeader(XLogReaderState *state,
 										 XLogRecPtr recptr, char *phdr);
 
+/* CRC-check an XLOG record */
+extern bool ValidXLogRecord(XLogReaderState *state, XLogRecord *record, XLogRecPtr recptr);
+
 /*
  * Error information from WALRead that both backend and frontend caller can
  * process.  Currently only errors from pg_pread can be reported.
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v4-0002-pg_walinspect-tests.patchapplication/octet-stream; name=v4-0002-pg_walinspect-tests.patchDownload
From 7e6e17d307ae875b7c169bc4523fcb1de7236072 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 9 Feb 2022 14:21:34 +0000
Subject: [PATCH v4] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 83 +++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 68 +++++++++++++++
 2 files changed, 151 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..fdcba5accc
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,83 @@
+CREATE EXTENSION pg_walinspect;
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+CREATE TABLE sample_tbl(col1 int, col2 int);
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL record start LSN
+SELECT pg_get_wal_records_info(:'wal_lsn1', '0/0'); -- ERROR
+ERROR:  invalid WAL record end LSN
+SELECT pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+ERROR:  WAL record start LSN must be less than end LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL record start LSN
+SELECT pg_get_wal_stats(:'wal_lsn1', '0/0'); -- ERROR
+ERROR:  invalid WAL record end LSN
+SELECT pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+ERROR:  WAL record start LSN must be less than end LSN
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn1') AS valid_wal_lsn1 \gset
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn2') AS valid_wal_lsn2 \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT record AS raw_wal_rec FROM pg_get_raw_wal_record(:'valid_wal_lsn1') \gset
+SELECT pg_verify_raw_wal_record(:'raw_wal_rec');
+ pg_verify_raw_wal_record 
+--------------------------
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..7d0c7398c8
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,68 @@
+CREATE EXTENSION pg_walinspect;
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+INSERT INTO sample_tbl SELECT i, i+1 FROM generate_series(1, 100) i;
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info(:'wal_lsn1', '0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats(:'wal_lsn1', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn1') AS valid_wal_lsn1 \gset
+
+SELECT pg_get_first_valid_wal_record_lsn(:'wal_lsn2') AS valid_wal_lsn2 \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+
+SELECT record AS raw_wal_rec FROM pg_get_raw_wal_record(:'valid_wal_lsn1') \gset
+
+SELECT pg_verify_raw_wal_record(:'raw_wal_rec');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v4-0003-pg_walinspect-docs.patchapplication/octet-stream; name=v4-0003-pg_walinspect-docs.patchDownload
From dda4f219d98851e80bd5d691c22204873a9940f9 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 9 Feb 2022 18:11:05 +0000
Subject: [PATCH v4] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 120 +++++++++++++++++++++++++++++++++
 3 files changed, 122 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index be9711c6f2..19614a42e1 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -130,6 +130,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index 328cd1f378..a2e8fd4a08 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -146,6 +146,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..a2c11d36b7
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,120 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging and analytical
+  purposes.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_monitor</literal> role. Access may be granted to others using
+  <command>GRANT</command>.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>pg_get_raw_wal_record(in_lsn pg_lsn, lsn OUT pg_lsn, record OUT bytea)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. Issues a warning if the given
+	   LSN wasn't a pointer to the start of a record and also wasn't a pointer
+	   to the beginning of a WAL segment file.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_first_valid_wal_record_lsn(in_lsn pg_lsn, lsn OUT pg_lsn)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Checks if given LSN points to a valid WAL record, if yes, just outputs
+	   it. Otherwise, gets the pointer to the start of a valid WAL record data
+	   of the given LSN. Issues a warning if the given LSN wasn't a pointer to
+	   the start of a record and also wasn't a pointer to the beginning of a WAL
+	   segment file.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_verify_raw_wal_record(record bytea, is_valid OUT boolean)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Checks correctness of given raw WAL record by verifying its CRC. Returns
+	   <literal>true</literal> if WAL record is valid, otherwise <literal>false</literal>.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_wal_record_info(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of given LSN. Issues a warning if the given
+	   LSN wasn't a pointer to the start of a record and also wasn't a pointer
+	   to the beginning of a WAL segment file.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_records_info(start_lsn pg_lsn, end_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. Issues a warning if the given
+      <replaceable>start_lsn</replaceable> wasn't a pointer to the start of a
+      record and also wasn't a pointer to the beginning of a WAL segment file.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_stats(start_lsn pg_lsn, end_lsn pg_lsn, resource_manager OUT text, count OUT int8, count_percentage OUT float4, record_size OUT int8, record_size_percentage OUT float4, fpi_size OUT int8, fpi_size_percentage OUT float4, combined_size OUT int8, combined_size_percentage OUT float4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each <replaceable>resource_manager</replaceable> type.
+      Issues a warning if the given <replaceable>start_lsn</replaceable> wasn't
+      a pointer to the start of a record and also wasn't a pointer to the
+      beginning of a WAL segment file.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+</sect1>
-- 
2.25.1

In reply to: Robert Haas (#23)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Sun, Feb 6, 2022 at 7:45 AM Robert Haas <robertmhaas@gmail.com> wrote:

For what it's worth, I am generally in favor of having something like
this in PostgreSQL. I think it's wrong of us to continue assuming that
everyone has command-line access. Even when that's true, it's not
necessarily convenient. If you choose to use a relational database,
you may be the sort of person who likes SQL. And if you are, you may
want to have the database tell you what's going on via SQL rather than
command-line tools or operating system utilities. Imagine if we didn't
have pg_stat_activity and you had to get that information by running a
separate binary. Would anyone like that? Why is this case any
different?

+1. An SQL interface is significantly easier to work with. Especially
because it can use the built-in LSN type, pg_lsn.

I don't find the slippery slope argument convincing. There aren't that
many other things that are like pg_waldump, but haven't already been
exposed via an SQL interface. Offhand, I can't think of any.

--
Peter Geoghegan

#26Andrew Dunstan
andrew@dunslane.net
In reply to: Robert Haas (#23)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On 2/6/22 10:45, Robert Haas wrote:

For what it's worth, I am generally in favor of having something like
this in PostgreSQL. I think it's wrong of us to continue assuming that
everyone has command-line access. Even when that's true, it's not
necessarily convenient. If you choose to use a relational database,
you may be the sort of person who likes SQL.

Almost completely off topic, but this reminded me of an incident about
30 years ago at my first gig as an SA/DBA. There was an application
programmer who insisted on loading a set of values from a text file into
a temp table (it was Ingres, anyone remember that?). Why? Because he
knew how to write "Select * from mytable order by mycol" but didn't know
how to drive the Unix sort utility at the command line. When I was
unable to restrain myself from smiling at this he got very angry and
yelled at me loudly.

So, yes, some people do like SQL and hate the command line.

cheers

andrew

--
Andrew Dunstan
EDB: https://www.enterprisedb.com

#27Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Andrew Dunstan (#26)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Feb 10, 2022 at 9:55 PM Peter Geoghegan <pg@bowt.ie> wrote:

On Sun, Feb 6, 2022 at 7:45 AM Robert Haas <robertmhaas@gmail.com> wrote:

For what it's worth, I am generally in favor of having something like
this in PostgreSQL. I think it's wrong of us to continue assuming that
everyone has command-line access. Even when that's true, it's not
necessarily convenient. If you choose to use a relational database,
you may be the sort of person who likes SQL. And if you are, you may
want to have the database tell you what's going on via SQL rather than
command-line tools or operating system utilities. Imagine if we didn't
have pg_stat_activity and you had to get that information by running a
separate binary. Would anyone like that? Why is this case any
different?

+1. An SQL interface is significantly easier to work with. Especially
because it can use the built-in LSN type, pg_lsn.

I don't find the slippery slope argument convincing. There aren't that
many other things that are like pg_waldump, but haven't already been
exposed via an SQL interface. Offhand, I can't think of any.

On Sat, Feb 12, 2022 at 4:03 AM Andrew Dunstan <andrew@dunslane.net> wrote:

Almost completely off topic, but this reminded me of an incident about
30 years ago at my first gig as an SA/DBA. There was an application
programmer who insisted on loading a set of values from a text file into
a temp table (it was Ingres, anyone remember that?). Why? Because he
knew how to write "Select * from mytable order by mycol" but didn't know
how to drive the Unix sort utility at the command line. When I was
unable to restrain myself from smiling at this he got very angry and
yelled at me loudly.

So, yes, some people do like SQL and hate the command line.

Thanks a lot for the comments. I'm looking forward to the review of
the latest v4 patches posted at [1]/messages/by-id/CALj2ACUS9+54QGPtUjk76dcYW-AMKp3hPe-U+pQo2-GpE4kjtA@mail.gmail.com.

[1]: /messages/by-id/CALj2ACUS9+54QGPtUjk76dcYW-AMKp3hPe-U+pQo2-GpE4kjtA@mail.gmail.com

Regards,
Bharath Rupireddy.

#28Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Bharath Rupireddy (#27)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Here are few comments:

+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{

Do we really need this function? I see that whenever the record is
read, we verify it. So could there be a scenario where any of these
functions would return an invalid WAL record?

--

Should we add a function that returns the pointer to the first and
probably the last WAL record in the WAL segment? This would help users
to inspect the wal records in the entire wal segment if they wish to
do so.

--

+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

--

+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10

We could probably have another variant of this function that would
work even if the end pointer is not specified, in which case the
default end pointer would be the last WAL record in the WAL segment.
Currently it mandates the use of an end pointer which slightly reduces
flexibility.

--

+
+/*
+ * Get the first valid raw WAL record lsn.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)

I think this function should return a pointer to the nearest valid WAL
record which can be the previous WAL record to the LSN entered by the
user or the next WAL record. If a user unknowingly enters an lsn that
does not exist then in such cases we should probably return the lsn of
the previous WAL record instead of hanging or waiting for the new WAL
record to arrive.

--

Another important point I would like to mention here is - have we made
an attempt to ensure that we try to share as much of code with
pg_waldump as possible so that if any changes happens in the
pg_waldump in future it gets applied here as well and additionally it
will also reduce the code duplication.

I haven't yet looked into the code in detail. I will have a look at it
asap. thanks.

--
With Regards,
Ashutosh Sharma.

On Sat, Feb 12, 2022 at 5:03 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Show quoted text

On Thu, Feb 10, 2022 at 9:55 PM Peter Geoghegan <pg@bowt.ie> wrote:

On Sun, Feb 6, 2022 at 7:45 AM Robert Haas <robertmhaas@gmail.com> wrote:

For what it's worth, I am generally in favor of having something like
this in PostgreSQL. I think it's wrong of us to continue assuming that
everyone has command-line access. Even when that's true, it's not
necessarily convenient. If you choose to use a relational database,
you may be the sort of person who likes SQL. And if you are, you may
want to have the database tell you what's going on via SQL rather than
command-line tools or operating system utilities. Imagine if we didn't
have pg_stat_activity and you had to get that information by running a
separate binary. Would anyone like that? Why is this case any
different?

+1. An SQL interface is significantly easier to work with. Especially
because it can use the built-in LSN type, pg_lsn.

I don't find the slippery slope argument convincing. There aren't that
many other things that are like pg_waldump, but haven't already been
exposed via an SQL interface. Offhand, I can't think of any.

On Sat, Feb 12, 2022 at 4:03 AM Andrew Dunstan <andrew@dunslane.net> wrote:

Almost completely off topic, but this reminded me of an incident about
30 years ago at my first gig as an SA/DBA. There was an application
programmer who insisted on loading a set of values from a text file into
a temp table (it was Ingres, anyone remember that?). Why? Because he
knew how to write "Select * from mytable order by mycol" but didn't know
how to drive the Unix sort utility at the command line. When I was
unable to restrain myself from smiling at this he got very angry and
yelled at me loudly.

So, yes, some people do like SQL and hate the command line.

Thanks a lot for the comments. I'm looking forward to the review of
the latest v4 patches posted at [1].

[1] /messages/by-id/CALj2ACUS9+54QGPtUjk76dcYW-AMKp3hPe-U+pQo2-GpE4kjtA@mail.gmail.com

Regards,
Bharath Rupireddy.

#29Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Ashutosh Sharma (#28)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Mon, Feb 14, 2022 at 8:32 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

Here are few comments:

Thanks for reviewing the patches.

+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{

Do we really need this function? I see that whenever the record is
read, we verify it. So could there be a scenario where any of these
functions would return an invalid WAL record?

Yes, this function can be useful. Imagine a case where raw WAL records
are fetched from one server using pg_get_wal_record_info and sent over
the network to another server (for fixing some of the corrupted data
pages or for whatever reasons), using pg_verify_raw_wal_record one can
verify authenticity.

Should we add a function that returns the pointer to the first and
probably the last WAL record in the WAL segment? This would help users
to inspect the wal records in the entire wal segment if they wish to
do so.

Good point. One can do this already with pg_get_wal_records_info and
pg_walfile_name_offset. Usually, the LSN format itself can give an
idea about the WAL file it is in.

postgres=# select lsn, pg_walfile_name_offset(lsn) from
pg_get_wal_records_info('0/5000000', '0/5FFFFFF') order by lsn asc
limit 1;
lsn | pg_walfile_name_offset
-----------+-------------------------------
0/5000038 | (000000010000000000000005,56)
(1 row)

postgres=# select lsn, pg_walfile_name_offset(lsn) from
pg_get_wal_records_info('0/5000000', '0/5FFFFFF') order by lsn desc
limit 1;
lsn | pg_walfile_name_offset
-----------+-------------------------------------
0/5FFFFC0 | (000000010000000000000005,16777152)
(1 row)

Having said that, we can always add a function or a view (with the
above sort of queries) to pg_walinspect - given an LSN can give the
valid start record in that wal file (by following previous lsn links)
and valid end record lsn. IMO, that's not required now, maybe later
once the initial version of pg_walinspect gets committed, as we
already have a way to achieve what we wanted here.

+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

Currently, pg_walinspect uses read_local_xlog_page which waits in the
while(1) loop if a future LSN is specified. As read_local_xlog_page is
an implementation of XLogPageReadCB, which doesn't have a wait/nowait
parameter, if we really need a wait/nowait mode behaviour, we need to
do extra things(either add a backend-level global wait variable, set
before XLogReadRecord, if set, read_local_xlog_page can just exit
without waiting and reset after the XLogReadRecord or add an extra
bool wait variable to XLogReaderState and use it in
read_local_xlog_page).

Another problem with the wait mode is - wait until when? Because we
don't want to wait forever by specifying a really really future LSN,
maybe you could think of adding a timeout (if the future LSN hasn't
generated the given timeout, then just return). As I said upthread, I
think all of these functions can be parked to future pg_walinspect
versions once it gets committed with most-useful functions as proposed
in the v4 patch set.

+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10

We could probably have another variant of this function that would
work even if the end pointer is not specified, in which case the
default end pointer would be the last WAL record in the WAL segment.
Currently it mandates the use of an end pointer which slightly reduces
flexibility.

Last WAL record in the WAL segment may not be of much use(one can
figure out the last valid WAL record in a wal file as mentioned
above), but the WAL records info till the latest current flush LSN of
the server would be a useful functionality. But that too, can be found
using something like "select lsn, prev_lsn, resource_manager from
pg_get_wal_records_info('0/8099568', pg_current_wal_lsn());"

+
+/*
+ * Get the first valid raw WAL record lsn.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)

I think this function should return a pointer to the nearest valid WAL
record which can be the previous WAL record to the LSN entered by the
user or the next WAL record. If a user unknowingly enters an lsn that
does not exist then in such cases we should probably return the lsn of
the previous WAL record instead of hanging or waiting for the new WAL
record to arrive.

Is it useful? If there's a strong reason, how about naming
pg_get_next_valid_wal_record_lsn returning the next valid wal record
LSN and pg_get_previous_valid_wal_record_lsn returning the previous
valid wal record LSN ? If you think having two functions is too much,
then, how about pg_get_first_valid_wal_record_lsn returning both the
next valid wal record LSN and its previous wal record LSN?

Another important point I would like to mention here is - have we made
an attempt to ensure that we try to share as much of code with
pg_waldump as possible so that if any changes happens in the
pg_waldump in future it gets applied here as well and additionally it
will also reduce the code duplication.

I tried, please have a look at the patch. Also, I added a note at the
beginning of pg_walinspect and pg_waldump to consider fixing
issues/changing the code in both the places also.

I haven't yet looked into the code in detail. I will have a look at it
asap. thanks.

That will be great.

Regards,
Bharath Rupireddy.

#30Robert Haas
robertmhaas@gmail.com
In reply to: Bharath Rupireddy (#29)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Tue, Feb 15, 2022 at 2:31 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{

Do we really need this function? I see that whenever the record is
read, we verify it. So could there be a scenario where any of these
functions would return an invalid WAL record?

Yes, this function can be useful. Imagine a case where raw WAL records
are fetched from one server using pg_get_wal_record_info and sent over
the network to another server (for fixing some of the corrupted data
pages or for whatever reasons), using pg_verify_raw_wal_record one can
verify authenticity.

As I also said before, and so did Greg, I think giving the user a way
to supply WAL records that we will then try to decode is never going
to be OK. It's going to be a recipe for security bugs and crash bugs,
and there's no compelling use case for it that I can see. I support
this patch set only to the extent that it decodes locally generated
WAL read directly from the WAL stream.

--
Robert Haas
EDB: http://www.enterprisedb.com

#31Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Robert Haas (#30)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Feb 16, 2022 at 1:57 AM Robert Haas <robertmhaas@gmail.com> wrote:

On Tue, Feb 15, 2022 at 2:31 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{

Do we really need this function? I see that whenever the record is
read, we verify it. So could there be a scenario where any of these
functions would return an invalid WAL record?

Yes, this function can be useful. Imagine a case where raw WAL records
are fetched from one server using pg_get_wal_record_info and sent over
the network to another server (for fixing some of the corrupted data
pages or for whatever reasons), using pg_verify_raw_wal_record one can
verify authenticity.

As I also said before, and so did Greg, I think giving the user a way
to supply WAL records that we will then try to decode is never going
to be OK. It's going to be a recipe for security bugs and crash bugs,
and there's no compelling use case for it that I can see. I support
this patch set only to the extent that it decodes locally generated
WAL read directly from the WAL stream.

Agreed, I will remove pg_verify_raw_wal_record function in the next
version of the patch set. Thanks.

Regards,
Bharath Rupireddy.

#32Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Bharath Rupireddy (#29)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Feb 16, 2022 at 1:01 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

On Mon, Feb 14, 2022 at 8:32 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

Here are few comments:

Thanks for reviewing the patches.

+/*
+ * Verify the authenticity of the given raw WAL record.
+ */
+Datum
+pg_verify_raw_wal_record(PG_FUNCTION_ARGS)
+{

Do we really need this function? I see that whenever the record is
read, we verify it. So could there be a scenario where any of these
functions would return an invalid WAL record?

Yes, this function can be useful. Imagine a case where raw WAL records
are fetched from one server using pg_get_wal_record_info and sent over
the network to another server (for fixing some of the corrupted data
pages or for whatever reasons), using pg_verify_raw_wal_record one can
verify authenticity.

I don't think that's the use case of this patch. Unless there is some
other valid reason, I would suggest you remove it.

Should we add a function that returns the pointer to the first and
probably the last WAL record in the WAL segment? This would help users
to inspect the wal records in the entire wal segment if they wish to
do so.

Good point. One can do this already with pg_get_wal_records_info and
pg_walfile_name_offset. Usually, the LSN format itself can give an
idea about the WAL file it is in.

postgres=# select lsn, pg_walfile_name_offset(lsn) from
pg_get_wal_records_info('0/5000000', '0/5FFFFFF') order by lsn asc
limit 1;
lsn | pg_walfile_name_offset
-----------+-------------------------------
0/5000038 | (000000010000000000000005,56)
(1 row)

postgres=# select lsn, pg_walfile_name_offset(lsn) from
pg_get_wal_records_info('0/5000000', '0/5FFFFFF') order by lsn desc
limit 1;
lsn | pg_walfile_name_offset
-----------+-------------------------------------
0/5FFFFC0 | (000000010000000000000005,16777152)
(1 row)

The workaround you are suggesting is not very user friendly and FYKI
pg_wal_records_info simply hangs at times when we specify the higher
and lower limit of lsn in a wal file.

To make things easier for the end users I would suggest we add a
function that can return a valid first and last lsn in a walfile. The
output of this function can be used to inspect the wal records in the
entire wal file if they wish to do so and I am sure they will. So, it
should be something like this:

select first_valid_lsn, last_valid_lsn from
pg_get_first_last_valid_wal_record('wal-segment-name');

And above function can directly be used with pg_get_wal_records_info() like

select pg_get_wal_records_info(pg_get_first_last_valid_wal_record('wal-segment'));

I think this is a pretty basic ASK that we expect to be present in the
module like this.

+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

Currently, pg_walinspect uses read_local_xlog_page which waits in the
while(1) loop if a future LSN is specified. As read_local_xlog_page is
an implementation of XLogPageReadCB, which doesn't have a wait/nowait
parameter, if we really need a wait/nowait mode behaviour, we need to
do extra things(either add a backend-level global wait variable, set
before XLogReadRecord, if set, read_local_xlog_page can just exit
without waiting and reset after the XLogReadRecord or add an extra
bool wait variable to XLogReaderState and use it in
read_local_xlog_page).

I am not asking to do any changes in the backend code. Please check -
how pg_waldump does this when a user requests to stop once the endptr
has reached. If not for all functions at least for a few functions we
can do this if it is doable.

+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10

We could probably have another variant of this function that would
work even if the end pointer is not specified, in which case the
default end pointer would be the last WAL record in the WAL segment.
Currently it mandates the use of an end pointer which slightly reduces
flexibility.

Last WAL record in the WAL segment may not be of much use(one can
figure out the last valid WAL record in a wal file as mentioned
above), but the WAL records info till the latest current flush LSN of
the server would be a useful functionality. But that too, can be found
using something like "select lsn, prev_lsn, resource_manager from
pg_get_wal_records_info('0/8099568', pg_current_wal_lsn());"

What if a user wants to inspect all the valid wal records from a
startptr (startlsn) and he doesn't know the endptr? Why should he/she
be mandated to get the endptr and supply it to this function? I don't
think we should force users to do that. I think this is again a very
basic ASK that can be done in this version itself. It is not at all
any advanced thing that we can think of doing in the future.

+
+/*
+ * Get the first valid raw WAL record lsn.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)

I think this function should return a pointer to the nearest valid WAL
record which can be the previous WAL record to the LSN entered by the
user or the next WAL record. If a user unknowingly enters an lsn that
does not exist then in such cases we should probably return the lsn of
the previous WAL record instead of hanging or waiting for the new WAL
record to arrive.

Is it useful?

It is useful in the same way as returning the next valid wal pointer
is. Why should a user wait for the next valid wal pointer to be
available instead the function should identify the previous valid wal
record and return it and put an appropriate message to the user.

If there's a strong reason, how about naming

pg_get_next_valid_wal_record_lsn returning the next valid wal record
LSN and pg_get_previous_valid_wal_record_lsn returning the previous
valid wal record LSN ? If you think having two functions is too much,
then, how about pg_get_first_valid_wal_record_lsn returning both the
next valid wal record LSN and its previous wal record LSN?

The latter one looks better.

--
With Regards,
Ashutosh Sharma.

#33Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Ashutosh Sharma (#32)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Feb 16, 2022 at 9:04 AM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

I don't think that's the use case of this patch. Unless there is some
other valid reason, I would suggest you remove it.

Removed the function pg_verify_raw_wal_record. Robert and Greg also
voted for removal upthread.

Should we add a function that returns the pointer to the first and
probably the last WAL record in the WAL segment? This would help users
to inspect the wal records in the entire wal segment if they wish to
do so.

Good point. One can do this already with pg_get_wal_records_info and
pg_walfile_name_offset. Usually, the LSN format itself can give an
idea about the WAL file it is in.

postgres=# select lsn, pg_walfile_name_offset(lsn) from
pg_get_wal_records_info('0/5000000', '0/5FFFFFF') order by lsn asc
limit 1;
lsn | pg_walfile_name_offset
-----------+-------------------------------
0/5000038 | (000000010000000000000005,56)
(1 row)

postgres=# select lsn, pg_walfile_name_offset(lsn) from
pg_get_wal_records_info('0/5000000', '0/5FFFFFF') order by lsn desc
limit 1;
lsn | pg_walfile_name_offset
-----------+-------------------------------------
0/5FFFFC0 | (000000010000000000000005,16777152)
(1 row)

The workaround you are suggesting is not very user friendly and FYKI
pg_wal_records_info simply hangs at times when we specify the higher
and lower limit of lsn in a wal file.

To make things easier for the end users I would suggest we add a
function that can return a valid first and last lsn in a walfile. The
output of this function can be used to inspect the wal records in the
entire wal file if they wish to do so and I am sure they will. So, it
should be something like this:

select first_valid_lsn, last_valid_lsn from
pg_get_first_last_valid_wal_record('wal-segment-name');

And above function can directly be used with pg_get_wal_records_info() like

select pg_get_wal_records_info(pg_get_first_last_valid_wal_record('wal-segment'));

I think this is a pretty basic ASK that we expect to be present in the
module like this.

Added a new function that returns the first and last valid WAL record
LSN of a given WAL file.

+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

Currently, pg_walinspect uses read_local_xlog_page which waits in the
while(1) loop if a future LSN is specified. As read_local_xlog_page is
an implementation of XLogPageReadCB, which doesn't have a wait/nowait
parameter, if we really need a wait/nowait mode behaviour, we need to
do extra things(either add a backend-level global wait variable, set
before XLogReadRecord, if set, read_local_xlog_page can just exit
without waiting and reset after the XLogReadRecord or add an extra
bool wait variable to XLogReaderState and use it in
read_local_xlog_page).

I am not asking to do any changes in the backend code. Please check -
how pg_waldump does this when a user requests to stop once the endptr
has reached. If not for all functions at least for a few functions we
can do this if it is doable.

I've added a new function read_local_xlog_page_2 (similar to
read_local_xlog_page but works in wait and no wait mode) and the
callers can specify whether to wait or not wait using private_data.
Actually, I wanted to use the private_data structure of
read_local_xlog_page but the logical decoding already has context as
private_data, that is why I had to have a new function. I know it
creates a bit of duplicate code, but its cleaner than using
backend-local variables or additional flags in XLogReaderState or
adding wait/no-wait boolean to page_read callback. Any other
suggestions are welcome here.

With this, I'm able to have wait/no wait versions for any functions.
But for now, I'm having wait/no wait for two functions
(pg_get_wal_records_info and pg_get_wal_stats) for which it makes more
sense.

+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10

We could probably have another variant of this function that would
work even if the end pointer is not specified, in which case the
default end pointer would be the last WAL record in the WAL segment.
Currently it mandates the use of an end pointer which slightly reduces
flexibility.

Last WAL record in the WAL segment may not be of much use(one can
figure out the last valid WAL record in a wal file as mentioned
above), but the WAL records info till the latest current flush LSN of
the server would be a useful functionality. But that too, can be found
using something like "select lsn, prev_lsn, resource_manager from
pg_get_wal_records_info('0/8099568', pg_current_wal_lsn());"

What if a user wants to inspect all the valid wal records from a
startptr (startlsn) and he doesn't know the endptr? Why should he/she
be mandated to get the endptr and supply it to this function? I don't
think we should force users to do that. I think this is again a very
basic ASK that can be done in this version itself. It is not at all
any advanced thing that we can think of doing in the future.

Agreed. Added new functions that emits wal records info/stats till the
end of the WAL at the moment.

+
+/*
+ * Get the first valid raw WAL record lsn.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)

I think this function should return a pointer to the nearest valid WAL
record which can be the previous WAL record to the LSN entered by the
user or the next WAL record. If a user unknowingly enters an lsn that
does not exist then in such cases we should probably return the lsn of
the previous WAL record instead of hanging or waiting for the new WAL
record to arrive.

Is it useful?

It is useful in the same way as returning the next valid wal pointer
is. Why should a user wait for the next valid wal pointer to be
available instead the function should identify the previous valid wal
record and return it and put an appropriate message to the user.

If there's a strong reason, how about naming

pg_get_next_valid_wal_record_lsn returning the next valid wal record
LSN and pg_get_previous_valid_wal_record_lsn returning the previous
valid wal record LSN ? If you think having two functions is too much,
then, how about pg_get_first_valid_wal_record_lsn returning both the
next valid wal record LSN and its previous wal record LSN?

The latter one looks better.

Modified.

Attaching v5 patch set, please review it further.

Regards,
Bharath Rupireddy.

Attachments:

v5-0001-pg_walinspect.patchapplication/x-patch; name=v5-0001-pg_walinspect.patchDownload
From 42e9f32a8579505adc38366a12c8c6d63f18fb83 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 25 Feb 2022 10:16:43 +0000
Subject: [PATCH v5] pg_walinspect

---
 contrib/Makefile                             |    1 +
 contrib/pg_walinspect/.gitignore             |    4 +
 contrib/pg_walinspect/Makefile               |   26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql |  146 +++
 contrib/pg_walinspect/pg_walinspect.c        | 1227 ++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |    5 +
 src/backend/access/transam/xlogreader.c      |   14 +-
 src/backend/access/transam/xlogutils.c       |  142 ++
 src/bin/pg_waldump/pg_waldump.c              |    5 +
 src/common/relpath.c                         |   18 +
 src/include/access/xlog.h                    |    2 +-
 src/include/access/xlog_internal.h           |    2 +-
 src/include/access/xlogreader.h              |    2 -
 src/include/access/xlogutils.h               |   13 +
 src/include/common/relpath.h                 |    1 +
 15 files changed, 1597 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index e3e221308b..705c6fc36b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -40,6 +40,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..d1828841f0
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,146 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_first_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_valid_wal_record_lsn(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_valid_wal_record_lsn'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_first_and_last_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_and_last_valid_wal_record_lsn(IN wal_file_name text,
+    OUT first_valid_lsn pg_lsn,
+    OUT last_valid_lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_and_last_valid_wal_record_lsn'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_first_and_last_valid_wal_record_lsn(text) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn, boolean) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean) TO pg_monitor;
+
+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..02ea21c860
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,1227 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+#define MAX_XLINFO_TYPES 16
+
+#define IsEndOfWALReached(state) \
+		(state->private_data != NULL && \
+		(((ReadLocalXLOGPage2Private *) xlogreader->private_data)->no_wait == true) && \
+		(((ReadLocalXLOGPage2Private *) xlogreader->private_data)->reached_end_of_wal == true))
+
+typedef struct Stats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+}	Stats;
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	Stats		rmgr_stats[RM_NEXT_ID];
+	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+}	XLogRecStats;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_first_and_last_valid_wal_record_lsn);
+
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record,
+											bool warning,
+											bool wait_for_wal);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls,
+							  uint32 ncols);
+static void StoreXLogRecordStats(XLogRecStats * stats,
+								 XLogReaderState *record);
+static void GetXLogSummaryStats(XLogRecStats * stats,
+								Tuplestorestate *tupstore, TupleDesc tupdesc,
+								Datum *values, bool *nulls,
+								uint32 ncols);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+								XLogRecPtr end_lsn, bool wait_for_wal);
+static void GetWALRecordsInfoInternal(FunctionCallInfo fcinfo,
+									  XLogRecPtr start_lsn, XLogRecPtr end_lsn,
+									  bool wait_for_wal);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record,
+					bool warning, bool wait_for_wal)
+{
+	XLogReaderState *xlogreader;
+	ReadLocalXLOGPage2Private *private_data = NULL;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	/*
+	 * When told let's not wait for WAL to be available. Fill in the
+	 * private_data so that the page_read callback can return whenever
+	 * end-of-WAL is reached.
+	 */
+	if (wait_for_wal == false)
+	{
+		private_data = (ReadLocalXLOGPage2Private *)
+						 palloc0(sizeof(ReadLocalXLOGPage2Private));
+
+		private_data->no_wait = true;
+
+		/*
+		 * page_read callback will set this to true when the end-of-WAL is
+		 * reached.
+		 */
+		private_data->reached_end_of_wal = false;
+	}
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page_2,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									(void *) private_data);
+
+	if (xlogreader == NULL)
+	{
+		if (wait_for_wal == false)
+			pfree(private_data);
+
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+	}
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		ereport(WARNING,
+				(errmsg("reached end of WAL")));
+
+		return xlogreader;
+	}
+	else if (XLogRecPtrIsInvalid(*first_record))
+	{
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+	}
+
+	/*
+	 * Display a message that we're skipping data if the given LSN wasn't a
+	 * pointer to the start of a record and also wasn't a pointer to the
+	 * beginning of a segment (e.g. we were used in file mode).
+	 */
+	if (warning && *first_record != lsn &&
+		XLogSegmentOffset(lsn, wal_segment_size) != 0)
+	{
+		ereport(WARNING,
+				(errmsg_plural("first record is after %X/%X, at %X/%X, skipping over %u byte",
+							   "first record is after %X/%X, at %X/%X, skipping over %u bytes",
+							   (*first_record - lsn),
+							   LSN_FORMAT_ARGS(lsn),
+							   LSN_FORMAT_ARGS(*first_record),
+							   (uint32) (*first_record - lsn))));
+	}
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	char	   *errormsg;
+	XLogRecord *record;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL &&
+		IsEndOfWALReached(xlogreader))
+	{
+		ereport(WARNING,
+				(errmsg("reached end of WAL")));
+
+		return record;
+	}
+	else if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get raw WAL record.
+ *
+ * XXX: Currently, this function will not wait if the future WAL LSN is
+ * provided. We can have a wait version of it by using
+ * ReadLocalXLOGPage2Private.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	   *raw_record;
+	uint32		rec_len;
+	char	   *raw_record_data;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool		nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int			i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true, false);
+
+	Assert(xlogreader);
+
+	if (XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	if (record == NULL &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = PointerGetDatum(raw_record);
+
+	Assert(i == PG_GET_RAW_WAL_RECORD_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_RAW_WAL_RECORD_COLS
+}
+
+/*
+ * Get first valid raw WAL record LSN following the given LSN.
+ *
+ * XXX: Currently, this function will not wait if the future WAL LSN is
+ * provided. We can have a wait version of it by using
+ * ReadLocalXLOGPage2Private.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+#define PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogRecPtr	prev_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS];
+	bool		nulls[PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS];
+	int			i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true, false);
+
+	Assert(xlogreader);
+
+	if (XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	if (record == NULL &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	prev_record = XLogRecGetPrev(xlogreader);
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	/*
+	 * Previous valid WAL record must be at an LSN lower than next valid WAL
+	 * record LSN. Otherwise, it is an indication of something wrong, so error
+	 * out.
+	 */
+	if (prev_record >= first_record)
+		ereport(ERROR,
+				(errcode(ERRCODE_DATA_CORRUPTED),
+				 errmsg("record with incorrect prev-link %X/%X at %X/%X",
+				 LSN_FORMAT_ARGS(prev_record),
+				 LSN_FORMAT_ARGS(first_record))));
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(prev_record);
+
+	Assert(i == PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS
+}
+
+/*
+ * Get first and last valid record LSN in a WAL file.
+ */
+Datum
+pg_get_first_and_last_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+#define PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS 2
+	char	   *fname;
+	uint32		tli;
+	XLogSegNo	segno;
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogRecPtr	last_record = InvalidXLogRecPtr;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS];
+	bool		nulls[PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS];
+	int			i = 0;
+
+	fname = text_to_cstring(PG_GETARG_TEXT_PP(0));
+
+	/* Validate input. */
+	if (!IsXLogFileName(fname))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL file name")));
+
+	/* Looks like an xlog file. Parse its position. */
+	XLogFromFileName(fname, &tli, &segno, wal_segment_size);
+
+	XLogSegNoOffsetToRecPtr(segno, 0, wal_segment_size, lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true, false);
+
+	Assert(xlogreader);
+
+	if (XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	for (;;)
+	{
+		XLogRecord	*record;
+		char	cfname[MAXFNAMELEN];
+
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		/* Exit loop if end-of-WAL is reached. */
+		if (record == NULL &&
+			IsEndOfWALReached(xlogreader))
+			break;
+
+		XLByteToSeg(xlogreader->currRecPtr, segno, wal_segment_size);
+		XLogFileName(cfname, tli, segno, wal_segment_size);
+
+		if (!IsXLogFileName(cfname))
+			continue;
+
+		if (strncmp(cfname, fname, XLOG_FNAME_LEN) == 0)
+		{
+			/*
+			 * Given WAL file name and current record's WAL file name matches.
+			 * The current record might be the last valid record.
+			 */
+			last_record = xlogreader->currRecPtr;
+		}
+		else
+		{
+			/*
+			 * Given WAL file name and current record's WAL file name doesn't
+			 * match, so exit the loop. last_record contains the last valid
+			 * record in the given WAL file.
+			 */
+			break;
+		}
+	}
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(last_record);
+
+	Assert(i == PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS
+}
+
+/*
+ * Calculate size of a record, split into !FPI and FPI parts.
+ */
+static void
+GetXLogRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
+{
+	int			block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data. It doesn't seem worth it to
+	 * add an accessor macro for this.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += record->blocks[block_id].bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32		rec_len;
+	uint32		fpi_len;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int			block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	   *data;
+	char	   *main_data;
+	uint32		main_data_len;
+	int			i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(rec_len);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+
+	initStringInfo(&rec_desc);
+
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+
+	pfree(temp.data);
+
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		}
+		else
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+		}
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = PointerGetDatum(data);
+	values[i++] = UInt32GetDatum(main_data_len);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get the WAL record info and data.
+ *
+ * XXX: Currently, this function will not wait if the future WAL LSN is
+ * provided. We can have a wait version of it by using
+ * ReadLocalXLOGPage2Private.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true, false);
+
+	Assert(xlogreader);
+
+	if (XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	if (record == NULL &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+					  PG_GET_WAL_RECORD_INFO_COLS);
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn, bool wait_for_wal)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true,
+									 wait_for_wal);
+
+	Assert(xlogreader);
+
+	if (wait_for_wal == false &&
+		XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+		tuplestore_end(tupstore);
+
+		return;
+	}
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		XLogRecord *record;
+
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		/* Exit loop if end-of-WAL is reached. */
+		if (record == NULL &&
+			IsEndOfWALReached(xlogreader))
+			break;
+
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+						  PG_GET_WAL_RECORDS_INFO_COLS);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function will wait if the future WAL LSN is provided and wait_for_wal
+ * is true. By default, it doesn't wait for the future WAL LSN, but emits a
+ * warning and returns the available info.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	bool		wait_for_wal;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+	wait_for_wal = PG_GETARG_BOOL(2);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	if (XLogRecPtrIsInvalid(end_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record end LSN")));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL record start LSN must be less than end LSN")));
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, end_lsn, wait_for_wal);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get info and data of all WAL records from start LSN till end-of-WAL.
+ *
+ * This function is similar to pg_get_wal_records_info except that it doesn't
+ * have end LSN as input and it doesn't wait for future WAL.
+ */
+Datum
+pg_get_wal_records_info_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, InvalidXLogRecPtr,
+							  false);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+static void
+StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record)
+{
+	RmgrId		rmid;
+	uint8		recid;
+	uint32		rec_len;
+	uint32		fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics. */
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double		n_pct;
+	double		rec_len_pct;
+	double		fpi_len_pct;
+	double		tot_len_pct;
+	int			i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+					TupleDesc tupdesc, Datum *values, bool *nulls,
+					uint32 ncols)
+{
+	int			ri;
+	uint64		total_count = 0;
+	uint64		total_rec_len = 0;
+	uint64		total_fpi_len = 0;
+	uint64		total_len = 0;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls, ncols);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+					XLogRecPtr end_lsn, bool wait_for_wal)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogRecStats stats;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_STATS_COLS];
+	bool		nulls[PG_GET_WAL_STATS_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true,
+									 wait_for_wal);
+
+	Assert(xlogreader);
+
+	if (wait_for_wal == false &&
+		XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+		tuplestore_end(tupstore);
+
+		return;
+	}
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		XLogRecord *record;
+
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		/* Exit loop if end-of-WAL is reached. */
+		if (record == NULL &&
+			IsEndOfWALReached(xlogreader))
+			break;
+
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		StoreXLogRecordStats(&stats, xlogreader);
+	}
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, tupstore, tupdesc, values, nulls,
+						PG_GET_WAL_STATS_COLS);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ *
+ * This function will wait if the future WAL LSN is provided and wait_for_wal
+ * is true. By default, it doesn't wait for the future WAL LSN, but emits a
+ * warning and returns the available info.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	bool		wait_for_wal;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+	wait_for_wal = PG_GETARG_BOOL(2);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	if (XLogRecPtrIsInvalid(end_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record end LSN")));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL record start LSN must be less than end LSN")));
+
+	GetWalStatsInternal(fcinfo, start_lsn, end_lsn, wait_for_wal);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get WAL stats from start LSN till end-of-WAL.
+ *
+ * This function is similar to pg_get_wal_stats except that it doesn't have end
+ * LSN as input and it doesn't wait for future WAL.
+ */
+Datum
+pg_get_wal_stats_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	GetWalStatsInternal(fcinfo, start_lsn, InvalidXLogRecPtr, false);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index 35029cf97d..4d217cabbe 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -956,13 +956,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1080,6 +1073,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 54d5f20734..4c33933249 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -968,6 +968,148 @@ read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 	return count;
 }
 
+/*
+ * XLogReaderRoutine->page_read callback for reading local xlog files
+ *
+ * This function is same as read_local_xlog_page except that it works in both
+ * wait and no wait mode. The callers can specify about waiting in private_data
+ * of XLogReaderState.
+ */
+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+					   int reqLen, XLogRecPtr targetRecPtr, char *cur_page)
+{
+	XLogRecPtr	read_upto,
+				loc;
+	TimeLineID	tli;
+	int			count;
+	WALReadError errinfo;
+	TimeLineID	currTLI;
+
+	loc = targetPagePtr + reqLen;
+
+	/* Loop waiting for xlog to be available if necessary */
+	while (1)
+	{
+		/*
+		 * Determine the limit of xlog we can currently read to, and what the
+		 * most recent timeline is.
+		 */
+		if (!RecoveryInProgress())
+			read_upto = GetFlushRecPtr(&currTLI);
+		else
+			read_upto = GetXLogReplayRecPtr(&currTLI);
+		tli = currTLI;
+
+		/*
+		 * Check which timeline to get the record from.
+		 *
+		 * We have to do it each time through the loop because if we're in
+		 * recovery as a cascading standby, the current timeline might've
+		 * become historical. We can't rely on RecoveryInProgress() because in
+		 * a standby configuration like
+		 *
+		 * A => B => C
+		 *
+		 * if we're a logical decoding session on C, and B gets promoted, our
+		 * timeline will change while we remain in recovery.
+		 *
+		 * We can't just keep reading from the old timeline as the last WAL
+		 * archive in the timeline will get renamed to .partial by
+		 * StartupXLOG().
+		 *
+		 * If that happens after our caller determined the TLI but before
+		 * we actually read the xlog page, we might still try to read from the
+		 * old (now renamed) segment and fail. There's not much we can do
+		 * about this, but it can only happen when we're a leaf of a cascading
+		 * standby whose primary gets promoted while we're decoding, so a
+		 * one-off ERROR isn't too bad.
+		 */
+		XLogReadDetermineTimeline(state, targetPagePtr, reqLen, tli);
+
+		if (state->currTLI == currTLI)
+		{
+
+			if (loc <= read_upto)
+				break;
+
+			/* Let's not wait for WAL to be available if indicated */
+			if (loc > read_upto &&
+				state->private_data != NULL)
+			{
+				ReadLocalXLOGPage2Private *private;
+
+				private = (ReadLocalXLOGPage2Private *) state->private_data;
+
+				if (private->no_wait == true)
+				{
+					private->reached_end_of_wal = true;
+					break;
+				}
+			}
+
+			CHECK_FOR_INTERRUPTS();
+			pg_usleep(1000L);
+		}
+		else
+		{
+			/*
+			 * We're on a historical timeline, so limit reading to the switch
+			 * point where we moved to the next timeline.
+			 *
+			 * We don't need to GetFlushRecPtr or GetXLogReplayRecPtr. We know
+			 * about the new timeline, so we must've received past the end of
+			 * it.
+			 */
+			read_upto = state->currTLIValidUntil;
+
+			/*
+			 * Setting tli to our wanted record's TLI is slightly wrong; the
+			 * page might begin on an older timeline if it contains a timeline
+			 * switch, since its xlog segment will have been copied from the
+			 * prior timeline. This is pretty harmless though, as nothing
+			 * cares so long as the timeline doesn't go backwards.  We should
+			 * read the page header instead; FIXME someday.
+			 */
+			tli = state->currTLI;
+
+			/* No need to wait on a historical timeline */
+			break;
+		}
+	}
+
+	if (targetPagePtr + XLOG_BLCKSZ <= read_upto)
+	{
+		/*
+		 * more than one block available; read only that block, have caller
+		 * come back if they need more.
+		 */
+		count = XLOG_BLCKSZ;
+	}
+	else if (targetPagePtr + reqLen > read_upto)
+	{
+		/* not enough data there */
+		return -1;
+	}
+	else
+	{
+		/* enough bytes available to satisfy the request */
+		count = read_upto - targetPagePtr;
+	}
+
+	/*
+	 * Even though we just determined how much of the page can be validly read
+	 * as 'count', read the whole page anyway. It's guaranteed to be
+	 * zero-padded up to the page boundary if it's incomplete.
+	 */
+	if (!WALRead(state, cur_page, targetPagePtr, XLOG_BLCKSZ, tli,
+				 &errinfo))
+		WALReadRaiseError(&errinfo);
+
+	/* number of valid bytes in the buffer */
+	return count;
+}
+
 /*
  * Backend-specific convenience code to handle read errors encountered by
  * WALRead().
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index a6251e1a96..f8d61b56ab 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 4b45ac64db..f34f228563 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 849954a8e5..38fd51bafa 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 477f0efe26..d62d6ce7f8 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
diff --git a/src/include/access/xlogutils.h b/src/include/access/xlogutils.h
index 64708949db..dd28640f07 100644
--- a/src/include/access/xlogutils.h
+++ b/src/include/access/xlogutils.h
@@ -75,6 +75,16 @@ typedef enum
 								 * need to be replayed) */
 } XLogRedoAction;
 
+/*
+ * private_data of read_local_xlog_page, that is, XLogReaderRoutine->page_read
+ * callback for reading local xlog files.
+ */
+typedef struct ReadLocalXLOGPage2Private
+{
+	bool no_wait;
+	bool reached_end_of_wal;
+} ReadLocalXLOGPage2Private;
+
 extern XLogRedoAction XLogReadBufferForRedo(XLogReaderState *record,
 											uint8 buffer_id, Buffer *buf);
 extern Buffer XLogInitBufferForRedo(XLogReaderState *record, uint8 block_id);
@@ -92,6 +102,9 @@ extern void FreeFakeRelcacheEntry(Relation fakerel);
 extern int	read_local_xlog_page(XLogReaderState *state,
 								 XLogRecPtr targetPagePtr, int reqLen,
 								 XLogRecPtr targetRecPtr, char *cur_page);
+extern int	read_local_xlog_page_2(XLogReaderState *state,
+								   XLogRecPtr targetPagePtr, int reqLen,
+								   XLogRecPtr targetRecPtr, char *cur_page);
 extern void wal_segment_open(XLogReaderState *state,
 							 XLogSegNo nextSegNo,
 							 TimeLineID *tli_p);
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v5-0002-pg_walinspect-tests.patchapplication/x-patch; name=v5-0002-pg_walinspect-tests.patchDownload
From 1d2c2fd51cf0ad8704782aab2f215dd8e37fcbc2 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 25 Feb 2022 06:49:42 +0000
Subject: [PATCH v5] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 122 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   |  86 ++++++++++++
 2 files changed, 208 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..ba09c1caab
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,122 @@
+CREATE EXTENSION pg_walinspect;
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+CREATE TABLE sample_tbl(col1 int, col2 int);
+INSERT INTO sample_tbl VALUES(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl VALUES(2, 3);
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL record start LSN
+SELECT pg_get_wal_records_info(:'wal_lsn1', '0/0'); -- ERROR
+ERROR:  invalid WAL record end LSN
+SELECT pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+ERROR:  WAL record start LSN must be less than end LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL record start LSN
+SELECT pg_get_wal_stats(:'wal_lsn1', '0/0'); -- ERROR
+ERROR:  invalid WAL record end LSN
+SELECT pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+ERROR:  WAL record start LSN must be less than end LSN
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT lsn AS valid_wal_lsn1 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1') \gset
+SELECT lsn AS valid_wal_lsn2 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn2') \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'valid_wal_lsn2');
+WARNING:  reached end of WAL
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'valid_wal_lsn2');
+WARNING:  reached end of WAL
+ ok 
+----
+ t
+(1 row)
+
+SELECT file_name AS wal_file_name FROM pg_walfile_name_offset(:'valid_wal_lsn1') \gset
+SELECT COUNT(*) = 1 AS ok FROM pg_switch_wal();
+ ok 
+----
+ t
+(1 row)
+
+INSERT INTO sample_tbl VALUES(3, 4);
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_and_last_valid_wal_record_lsn(:'wal_file_name');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for functions with future WAL
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', 'FFFFFFFF/FFFFFFFF'); -- WARNING
+WARNING:  reached end of WAL
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', 'FFFFFFFF/FFFFFFFF'); -- WARNING
+WARNING:  reached end of WAL
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..5c4253787d
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,86 @@
+CREATE EXTENSION pg_walinspect;
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+INSERT INTO sample_tbl VALUES(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl VALUES(2, 3);
+
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info(:'wal_lsn1', '0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats(:'wal_lsn1', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+
+SELECT lsn AS valid_wal_lsn1 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1') \gset
+
+SELECT lsn AS valid_wal_lsn2 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn2') \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'valid_wal_lsn2');
+
+SELECT file_name AS wal_file_name FROM pg_walfile_name_offset(:'valid_wal_lsn1') \gset
+
+SELECT COUNT(*) = 1 AS ok FROM pg_switch_wal();
+
+INSERT INTO sample_tbl VALUES(3, 4);
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_and_last_valid_wal_record_lsn(:'wal_file_name');
+
+-- ===================================================================
+-- tests for functions with future WAL
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', 'FFFFFFFF/FFFFFFFF'); -- WARNING
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', 'FFFFFFFF/FFFFFFFF'); -- WARNING
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v5-0003-pg_walinspect-docs.patchapplication/x-patch; name=v5-0003-pg_walinspect-docs.patchDownload
From d1f3c3a295b842fbf6a3a9a040cfbd33e52d7333 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 25 Feb 2022 07:30:49 +0000
Subject: [PATCH v5] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 165 +++++++++++++++++++++++++++++++++
 3 files changed, 167 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index be9711c6f2..19614a42e1 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -130,6 +130,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index 328cd1f378..a2e8fd4a08 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -146,6 +146,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..fa9f1d9b6c
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,165 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging and analytical
+  purposes.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_monitor</literal> role. Access may be granted to others using
+  <command>GRANT</command>.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>pg_get_raw_wal_record(in_lsn pg_lsn, lsn OUT pg_lsn, record OUT bytea)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. Issues a warning if the given
+      LSN wasn't a pointer to the start of a record and also wasn't a pointer
+      to the beginning of a WAL segment file. This function will not wait if
+      the future WAL LSN is provided, instead emits a warning and returns an
+      empty row.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_first_valid_wal_record_lsn(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets first and previous valid WAL record LSNs of the given LSN. Issues
+      a warning if the given LSN wasn't a pointer to the start of a record and
+      also wasn't a pointer to the beginning of a WAL segment file. This
+      function will not wait if the future WAL LSN is provided, instead emits
+      a warning and returns an empty row.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_first_and_last_valid_wal_record_lsn(wal_file_name text, first_valid_lsn OUT pg_lsn, last_valid_lsn OUT pg_lsn)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets first and last valid WAL record LSNs of the given WAL file.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_wal_record_info(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of the given LSN. Issues a warning if the
+      given LSN wasn't a pointer to the start of a record and also wasn't a
+      pointer to the beginning of a WAL segment file. This function will not
+      wait if the future WAL LSN is provided, instead emits a warning and
+      returns an empty row.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_records_info(start_lsn pg_lsn, end_lsn pg_lsn, wait_for_wal boolean DEFAULT false, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. Issues a warning if the given
+      <replaceable>start_lsn</replaceable> wasn't a pointer to the start of a
+      record and also wasn't a pointer to the beginning of a WAL segment file.
+      This function will wait if the future WAL LSN is provided when
+      <replaceable>wait_for_wal</replaceable> is passed as <literal>true</literal>.
+      If <replaceable>wait_for_wal</replaceable> is <literal>false</literal>
+      (default value), it will not wait for the future WAL, instead emits a
+      warning and returns rows for whatever available WAL records.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_records_info_till_end_of_wal(start_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL. This function is
+      similar to <literal>pg_get_wal_records_info</literal> except that it
+      doesn't have <replaceable>end_lsn</replaceable> as input and it doesn't
+      wait for the future WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_stats(start_lsn pg_lsn, end_lsn pg_lsn, wait_for_wal boolean DEFAULT false, resource_manager OUT text, count OUT int8, count_percentage OUT float4, record_size OUT int8, record_size_percentage OUT float4, fpi_size OUT int8, fpi_size_percentage OUT float4, combined_size OUT int8, combined_size_percentage OUT float4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each <replaceable>resource_manager</replaceable>
+      type. Issues a warning if the given <replaceable>start_lsn</replaceable>
+      wasn't a pointer to the start of a record and also wasn't a pointer to
+      the beginning of a WAL segment file. This function will wait if the
+      future WAL LSN is provided when <replaceable>wait_for_wal</replaceable>
+      is passed as <literal>true</literal>. If <replaceable>wait_for_wal</replaceable>
+      is <literal>false</literal> (default value), it will not wait for the
+      future WAL, instead emits a warning and returns rows for whatever
+      available WAL records.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_stats_till_end_of_wal(start_lsn pg_lsn, end_lsn pg_lsn, resource_manager OUT text, count OUT int8, count_percentage OUT float4, record_size OUT int8, record_size_percentage OUT float4, fpi_size OUT int8, fpi_size_percentage OUT float4, combined_size OUT int8, combined_size_percentage OUT float4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL. This function is
+      similar to <literal>pg_get_wal_stats</literal> except that it doesn't
+      have <replaceable>end_lsn</replaceable> as input and it doesn't wait for
+      the future WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#34Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Bharath Rupireddy (#33)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Some review comments on v5 patch (v5-0001-pg_walinspect.patch)

+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT lsn pg_lsn,

What does the wait_for_wal flag mean here when one has already
specified the start and end lsn? AFAIU, If a user has specified a
start and stop LSN, it means that the user knows the extent to which
he/she wants to display the WAL records in which case we need to stop
once the end lsn has reached . So what is the meaning of wait_for_wal
flag? Does it look sensible to have the wait_for_wal flag here? To me
it doesn't.

==

+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,

Why is this function required? Is pg_get_wal_records_info() alone not
enough? I think it is. See if we can make end_lsn optional in
pg_get_wal_records_info() and lets just have it alone. I think it can
do the job of pg_get_wal_records_info_till_end_of_wal function.

==

+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,

Above comment applies to this function as well. Isn't pg_get_wal_stats() enough?

==

+                       if (loc <= read_upto)
+                               break;
+
+                       /* Let's not wait for WAL to be available if
indicated */
+                       if (loc > read_upto &&
+                               state->private_data != NULL)
+                       {

Why loc > read_upto? The first if condition is (loc <= read_upto)
followed by the second if condition - (loc > read_upto). Is the first
if condition (loc <= read_upto) not enough to indicate that loc >
read_upto?

==

+#define IsEndOfWALReached(state) \
+               (state->private_data != NULL && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->no_wait == true) && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))

I think we should either use state or xlogreader. First line says
state->private_data and second line xlogreader->private_data.

==

+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))
+

There is a new patch coming to make the end of WAL messages less
scary. It introduces the EOW flag in xlogreaderstate maybe we can use
that instead of introducing new flags in private area to represent the
end of WAL.

==

+/*
+ * XLogReaderRoutine->page_read callback for reading local xlog files
+ *
+ * This function is same as read_local_xlog_page except that it works in both
+ * wait and no wait mode. The callers can specify about waiting in private_data
+ * of XLogReaderState.
+ */
+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+                                          int reqLen, XLogRecPtr
targetRecPtr, char *cur_page)
+{
+       XLogRecPtr      read_upto,

Do we really need this function? Can't we make use of an existing WAL
reader function - read_local_xlog_page()?

--
With Regards,
Ashutosh Sharma.

On Fri, Feb 25, 2022 at 4:33 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Show quoted text

On Wed, Feb 16, 2022 at 9:04 AM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

I don't think that's the use case of this patch. Unless there is some
other valid reason, I would suggest you remove it.

Removed the function pg_verify_raw_wal_record. Robert and Greg also
voted for removal upthread.

Should we add a function that returns the pointer to the first and
probably the last WAL record in the WAL segment? This would help users
to inspect the wal records in the entire wal segment if they wish to
do so.

Good point. One can do this already with pg_get_wal_records_info and
pg_walfile_name_offset. Usually, the LSN format itself can give an
idea about the WAL file it is in.

postgres=# select lsn, pg_walfile_name_offset(lsn) from
pg_get_wal_records_info('0/5000000', '0/5FFFFFF') order by lsn asc
limit 1;
lsn | pg_walfile_name_offset
-----------+-------------------------------
0/5000038 | (000000010000000000000005,56)
(1 row)

postgres=# select lsn, pg_walfile_name_offset(lsn) from
pg_get_wal_records_info('0/5000000', '0/5FFFFFF') order by lsn desc
limit 1;
lsn | pg_walfile_name_offset
-----------+-------------------------------------
0/5FFFFC0 | (000000010000000000000005,16777152)
(1 row)

The workaround you are suggesting is not very user friendly and FYKI
pg_wal_records_info simply hangs at times when we specify the higher
and lower limit of lsn in a wal file.

To make things easier for the end users I would suggest we add a
function that can return a valid first and last lsn in a walfile. The
output of this function can be used to inspect the wal records in the
entire wal file if they wish to do so and I am sure they will. So, it
should be something like this:

select first_valid_lsn, last_valid_lsn from
pg_get_first_last_valid_wal_record('wal-segment-name');

And above function can directly be used with pg_get_wal_records_info() like

select pg_get_wal_records_info(pg_get_first_last_valid_wal_record('wal-segment'));

I think this is a pretty basic ASK that we expect to be present in the
module like this.

Added a new function that returns the first and last valid WAL record
LSN of a given WAL file.

+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

Currently, pg_walinspect uses read_local_xlog_page which waits in the
while(1) loop if a future LSN is specified. As read_local_xlog_page is
an implementation of XLogPageReadCB, which doesn't have a wait/nowait
parameter, if we really need a wait/nowait mode behaviour, we need to
do extra things(either add a backend-level global wait variable, set
before XLogReadRecord, if set, read_local_xlog_page can just exit
without waiting and reset after the XLogReadRecord or add an extra
bool wait variable to XLogReaderState and use it in
read_local_xlog_page).

I am not asking to do any changes in the backend code. Please check -
how pg_waldump does this when a user requests to stop once the endptr
has reached. If not for all functions at least for a few functions we
can do this if it is doable.

I've added a new function read_local_xlog_page_2 (similar to
read_local_xlog_page but works in wait and no wait mode) and the
callers can specify whether to wait or not wait using private_data.
Actually, I wanted to use the private_data structure of
read_local_xlog_page but the logical decoding already has context as
private_data, that is why I had to have a new function. I know it
creates a bit of duplicate code, but its cleaner than using
backend-local variables or additional flags in XLogReaderState or
adding wait/no-wait boolean to page_read callback. Any other
suggestions are welcome here.

With this, I'm able to have wait/no wait versions for any functions.
But for now, I'm having wait/no wait for two functions
(pg_get_wal_records_info and pg_get_wal_stats) for which it makes more
sense.

+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10

We could probably have another variant of this function that would
work even if the end pointer is not specified, in which case the
default end pointer would be the last WAL record in the WAL segment.
Currently it mandates the use of an end pointer which slightly reduces
flexibility.

Last WAL record in the WAL segment may not be of much use(one can
figure out the last valid WAL record in a wal file as mentioned
above), but the WAL records info till the latest current flush LSN of
the server would be a useful functionality. But that too, can be found
using something like "select lsn, prev_lsn, resource_manager from
pg_get_wal_records_info('0/8099568', pg_current_wal_lsn());"

What if a user wants to inspect all the valid wal records from a
startptr (startlsn) and he doesn't know the endptr? Why should he/she
be mandated to get the endptr and supply it to this function? I don't
think we should force users to do that. I think this is again a very
basic ASK that can be done in this version itself. It is not at all
any advanced thing that we can think of doing in the future.

Agreed. Added new functions that emits wal records info/stats till the
end of the WAL at the moment.

+
+/*
+ * Get the first valid raw WAL record lsn.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)

I think this function should return a pointer to the nearest valid WAL
record which can be the previous WAL record to the LSN entered by the
user or the next WAL record. If a user unknowingly enters an lsn that
does not exist then in such cases we should probably return the lsn of
the previous WAL record instead of hanging or waiting for the new WAL
record to arrive.

Is it useful?

It is useful in the same way as returning the next valid wal pointer
is. Why should a user wait for the next valid wal pointer to be
available instead the function should identify the previous valid wal
record and return it and put an appropriate message to the user.

If there's a strong reason, how about naming

pg_get_next_valid_wal_record_lsn returning the next valid wal record
LSN and pg_get_previous_valid_wal_record_lsn returning the previous
valid wal record LSN ? If you think having two functions is too much,
then, how about pg_get_first_valid_wal_record_lsn returning both the
next valid wal record LSN and its previous wal record LSN?

The latter one looks better.

Modified.

Attaching v5 patch set, please review it further.

Regards,
Bharath Rupireddy.

#35Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Ashutosh Sharma (#34)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Mar 2, 2022 at 8:12 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

Some review comments on v5 patch (v5-0001-pg_walinspect.patch)

Thanks for reviewing.

+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT lsn pg_lsn,

What does the wait_for_wal flag mean here when one has already
specified the start and end lsn? AFAIU, If a user has specified a
start and stop LSN, it means that the user knows the extent to which
he/she wants to display the WAL records in which case we need to stop
once the end lsn has reached . So what is the meaning of wait_for_wal
flag? Does it look sensible to have the wait_for_wal flag here? To me
it doesn't.

Users can always specify a future end_lsn and set wait_for_wal to
true, then the pg_get_wal_records_info/pg_get_wal_stats functions can
wait for the WAL. IMO, this is useful. If you remember you were okay
with wait/nowait versions for some of the functions upthread [1]/messages/by-id/CAE9k0P=9SReU_613TXytZmpwL3ZRpnC5zrf96UoNCATKpK-UxQ@mail.gmail.com +PG_FUNCTION_INFO_V1(pg_get_raw_wal_record); +PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn); +PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record); +PG_FUNCTION_INFO_V1(pg_get_wal_record_info); +PG_FUNCTION_INFO_V1(pg_get_wal_records_info);. I'm
not going to retain this behaviour for both
pg_get_wal_records_info/pg_get_wal_stats as it is similar to
pg_waldump's --follow option.

==

+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,

Why is this function required? Is pg_get_wal_records_info() alone not
enough? I think it is. See if we can make end_lsn optional in
pg_get_wal_records_info() and lets just have it alone. I think it can
do the job of pg_get_wal_records_info_till_end_of_wal function.

==

+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,

Above comment applies to this function as well. Isn't pg_get_wal_stats() enough?

I'm doing the following input validations for these functions to not
cause any issues with invalid LSN. If I were to have the default value
for end_lsn as 0/0, I can't perform input validations right? That is
the reason I'm having separate functions {pg_get_wal_records_info,
pg_get_wal_stats}_till_end_of_wal() versions.

/* Validate input. */
if (XLogRecPtrIsInvalid(start_lsn))
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("invalid WAL record start LSN")));

if (XLogRecPtrIsInvalid(end_lsn))
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("invalid WAL record end LSN")));

if (start_lsn >= end_lsn)
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("WAL record start LSN must be less than end LSN")));

==

+                       if (loc <= read_upto)
+                               break;
+
+                       /* Let's not wait for WAL to be available if
indicated */
+                       if (loc > read_upto &&
+                               state->private_data != NULL)
+                       {

Why loc > read_upto? The first if condition is (loc <= read_upto)
followed by the second if condition - (loc > read_upto). Is the first
if condition (loc <= read_upto) not enough to indicate that loc >
read_upto?

Yeah, that's unnecessary, I improved the comment there and removed loc

read_upto.

==

+#define IsEndOfWALReached(state) \
+               (state->private_data != NULL && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->no_wait == true) && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))

I think we should either use state or xlogreader. First line says
state->private_data and second line xlogreader->private_data.

I've changed it to use state instead of xlogreader.

==

+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))
+

There is a new patch coming to make the end of WAL messages less
scary. It introduces the EOW flag in xlogreaderstate maybe we can use
that instead of introducing new flags in private area to represent the
end of WAL.

Yeah that would be great. But we never know which one gets committed
first. Until then it's not good to have dependencies on two "on-going"
patches. Later, we can change.

==

+/*
+ * XLogReaderRoutine->page_read callback for reading local xlog files
+ *
+ * This function is same as read_local_xlog_page except that it works in both
+ * wait and no wait mode. The callers can specify about waiting in private_data
+ * of XLogReaderState.
+ */
+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+                                          int reqLen, XLogRecPtr
targetRecPtr, char *cur_page)
+{
+       XLogRecPtr      read_upto,

Do we really need this function? Can't we make use of an existing WAL
reader function - read_local_xlog_page()?

I clearly explained the reasons upthread [2]/messages/by-id/CALj2ACUtqWX95uAj2VNJED0PnixEeQ=0MEzpouLi+zd_iTugRA@mail.gmail.com I've added a new function read_local_xlog_page_2 (similar to read_local_xlog_page but works in wait and no wait mode) and the callers can specify whether to wait or not wait using private_data. Actually, I wanted to use the private_data structure of read_local_xlog_page but the logical decoding already has context as private_data, that is why I had to have a new function. I know it creates a bit of duplicate code, but its cleaner than using backend-local variables or additional flags in XLogReaderState or adding wait/no-wait boolean to page_read callback. Any other suggestions are welcome here.. Please let me know if
you have more thoughts/doubts here, we can connect offlist.

Attaching v6 patch set with above review comments addressed. Please
review it further.

[1] https://www.postgresql.org/message-id/CAE9k0P%3D9SReU_613TXytZmpwL3ZRpnC5zrf96UoNCATKpK-UxQ%40mail.gmail.com
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

[2]: /messages/by-id/CALj2ACUtqWX95uAj2VNJED0PnixEeQ=0MEzpouLi+zd_iTugRA@mail.gmail.com I've added a new function read_local_xlog_page_2 (similar to read_local_xlog_page but works in wait and no wait mode) and the callers can specify whether to wait or not wait using private_data. Actually, I wanted to use the private_data structure of read_local_xlog_page but the logical decoding already has context as private_data, that is why I had to have a new function. I know it creates a bit of duplicate code, but its cleaner than using backend-local variables or additional flags in XLogReaderState or adding wait/no-wait boolean to page_read callback. Any other suggestions are welcome here.
I've added a new function read_local_xlog_page_2 (similar to
read_local_xlog_page but works in wait and no wait mode) and the
callers can specify whether to wait or not wait using private_data.
Actually, I wanted to use the private_data structure of
read_local_xlog_page but the logical decoding already has context as
private_data, that is why I had to have a new function. I know it
creates a bit of duplicate code, but its cleaner than using
backend-local variables or additional flags in XLogReaderState or
adding wait/no-wait boolean to page_read callback. Any other
suggestions are welcome here.

With this, I'm able to have wait/no wait versions for any functions.
But for now, I'm having wait/no wait for two functions
(pg_get_wal_records_info and pg_get_wal_stats) for which it makes more
sense.

Regards,
Bharath Rupireddy.

Attachments:

v6-0001-pg_walinspect.patchapplication/octet-stream; name=v6-0001-pg_walinspect.patchDownload
From b62e2e2f55ed68be525ca0012be04c4797669ed7 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 2 Mar 2022 17:03:52 +0000
Subject: [PATCH v6] pg_walinspect

---
 contrib/Makefile                             |    1 +
 contrib/pg_walinspect/.gitignore             |    4 +
 contrib/pg_walinspect/Makefile               |   26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql |  146 +++
 contrib/pg_walinspect/pg_walinspect.c        | 1227 ++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |    5 +
 src/backend/access/transam/xlogreader.c      |   14 +-
 src/backend/access/transam/xlogutils.c       |  144 ++
 src/bin/pg_waldump/pg_waldump.c              |    5 +
 src/common/relpath.c                         |   18 +
 src/include/access/xlog.h                    |    2 +-
 src/include/access/xlog_internal.h           |    2 +-
 src/include/access/xlogreader.h              |    2 -
 src/include/access/xlogutils.h               |   13 +
 src/include/common/relpath.h                 |    1 +
 15 files changed, 1599 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index e3e221308b..705c6fc36b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -40,6 +40,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..d1828841f0
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,146 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_first_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_valid_wal_record_lsn(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_valid_wal_record_lsn'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_first_and_last_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_and_last_valid_wal_record_lsn(IN wal_file_name text,
+    OUT first_valid_lsn pg_lsn,
+    OUT last_valid_lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_and_last_valid_wal_record_lsn'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_first_and_last_valid_wal_record_lsn(text) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn, boolean) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean) TO pg_monitor;
+
+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..e16f6c41fc
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,1227 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+#define MAX_XLINFO_TYPES 16
+
+#define IsEndOfWALReached(state) \
+		(state->private_data != NULL && \
+		(((ReadLocalXLOGPage2Private *) state->private_data)->no_wait == true) && \
+		(((ReadLocalXLOGPage2Private *) state->private_data)->reached_end_of_wal == true))
+
+typedef struct Stats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+}	Stats;
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	Stats		rmgr_stats[RM_NEXT_ID];
+	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+}	XLogRecStats;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_first_and_last_valid_wal_record_lsn);
+
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record,
+											bool warning,
+											bool wait_for_wal);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls,
+							  uint32 ncols);
+static void StoreXLogRecordStats(XLogRecStats * stats,
+								 XLogReaderState *record);
+static void GetXLogSummaryStats(XLogRecStats * stats,
+								Tuplestorestate *tupstore, TupleDesc tupdesc,
+								Datum *values, bool *nulls,
+								uint32 ncols);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+								XLogRecPtr end_lsn, bool wait_for_wal);
+static void GetWALRecordsInfoInternal(FunctionCallInfo fcinfo,
+									  XLogRecPtr start_lsn, XLogRecPtr end_lsn,
+									  bool wait_for_wal);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record,
+					bool warning, bool wait_for_wal)
+{
+	XLogReaderState *xlogreader;
+	ReadLocalXLOGPage2Private *private_data = NULL;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	/*
+	 * When told let's not wait for WAL to be available. Fill in the
+	 * private_data so that the page_read callback can return whenever
+	 * end-of-WAL is reached.
+	 */
+	if (wait_for_wal == false)
+	{
+		private_data = (ReadLocalXLOGPage2Private *)
+						 palloc0(sizeof(ReadLocalXLOGPage2Private));
+
+		private_data->no_wait = true;
+
+		/*
+		 * page_read callback will set this to true when the end-of-WAL is
+		 * reached.
+		 */
+		private_data->reached_end_of_wal = false;
+	}
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page_2,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									(void *) private_data);
+
+	if (xlogreader == NULL)
+	{
+		if (wait_for_wal == false)
+			pfree(private_data);
+
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+	}
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		ereport(WARNING,
+				(errmsg("reached end of WAL")));
+
+		return xlogreader;
+	}
+	else if (XLogRecPtrIsInvalid(*first_record))
+	{
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+	}
+
+	/*
+	 * Display a message that we're skipping data if the given LSN wasn't a
+	 * pointer to the start of a record and also wasn't a pointer to the
+	 * beginning of a segment (e.g. we were used in file mode).
+	 */
+	if (warning && *first_record != lsn &&
+		XLogSegmentOffset(lsn, wal_segment_size) != 0)
+	{
+		ereport(WARNING,
+				(errmsg_plural("first record is after %X/%X, at %X/%X, skipping over %u byte",
+							   "first record is after %X/%X, at %X/%X, skipping over %u bytes",
+							   (*first_record - lsn),
+							   LSN_FORMAT_ARGS(lsn),
+							   LSN_FORMAT_ARGS(*first_record),
+							   (uint32) (*first_record - lsn))));
+	}
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	char	   *errormsg;
+	XLogRecord *record;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL &&
+		IsEndOfWALReached(xlogreader))
+	{
+		ereport(WARNING,
+				(errmsg("reached end of WAL")));
+
+		return record;
+	}
+	else if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get raw WAL record.
+ *
+ * XXX: Currently, this function will not wait if the future WAL LSN is
+ * provided. We can have a wait version of it by using
+ * ReadLocalXLOGPage2Private.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	   *raw_record;
+	uint32		rec_len;
+	char	   *raw_record_data;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool		nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int			i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true, false);
+
+	Assert(xlogreader);
+
+	if (XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	if (record == NULL &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = PointerGetDatum(raw_record);
+
+	Assert(i == PG_GET_RAW_WAL_RECORD_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_RAW_WAL_RECORD_COLS
+}
+
+/*
+ * Get first valid raw WAL record LSN following the given LSN.
+ *
+ * XXX: Currently, this function will not wait if the future WAL LSN is
+ * provided. We can have a wait version of it by using
+ * ReadLocalXLOGPage2Private.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+#define PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogRecPtr	prev_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS];
+	bool		nulls[PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS];
+	int			i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true, false);
+
+	Assert(xlogreader);
+
+	if (XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	if (record == NULL &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	prev_record = XLogRecGetPrev(xlogreader);
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	/*
+	 * Previous valid WAL record must be at an LSN lower than next valid WAL
+	 * record LSN. Otherwise, it is an indication of something wrong, so error
+	 * out.
+	 */
+	if (prev_record >= first_record)
+		ereport(ERROR,
+				(errcode(ERRCODE_DATA_CORRUPTED),
+				 errmsg("record with incorrect prev-link %X/%X at %X/%X",
+				 LSN_FORMAT_ARGS(prev_record),
+				 LSN_FORMAT_ARGS(first_record))));
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(prev_record);
+
+	Assert(i == PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS
+}
+
+/*
+ * Get first and last valid record LSN in a WAL file.
+ */
+Datum
+pg_get_first_and_last_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+#define PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS 2
+	char	   *fname;
+	uint32		tli;
+	XLogSegNo	segno;
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogRecPtr	last_record = InvalidXLogRecPtr;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS];
+	bool		nulls[PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS];
+	int			i = 0;
+
+	fname = text_to_cstring(PG_GETARG_TEXT_PP(0));
+
+	/* Validate input. */
+	if (!IsXLogFileName(fname))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL file name")));
+
+	/* Looks like an xlog file. Parse its position. */
+	XLogFromFileName(fname, &tli, &segno, wal_segment_size);
+
+	XLogSegNoOffsetToRecPtr(segno, 0, wal_segment_size, lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true, false);
+
+	Assert(xlogreader);
+
+	if (XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	for (;;)
+	{
+		XLogRecord	*record;
+		char	cfname[MAXFNAMELEN];
+
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		/* Exit loop if end-of-WAL is reached. */
+		if (record == NULL &&
+			IsEndOfWALReached(xlogreader))
+			break;
+
+		XLByteToSeg(xlogreader->currRecPtr, segno, wal_segment_size);
+		XLogFileName(cfname, tli, segno, wal_segment_size);
+
+		if (!IsXLogFileName(cfname))
+			continue;
+
+		if (strncmp(cfname, fname, XLOG_FNAME_LEN) == 0)
+		{
+			/*
+			 * Given WAL file name and current record's WAL file name matches.
+			 * The current record might be the last valid record.
+			 */
+			last_record = xlogreader->currRecPtr;
+		}
+		else
+		{
+			/*
+			 * Given WAL file name and current record's WAL file name doesn't
+			 * match, so exit the loop. last_record contains the last valid
+			 * record in the given WAL file.
+			 */
+			break;
+		}
+	}
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(last_record);
+
+	Assert(i == PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_FIRST_AND_LAST_VALID_WAL_RECORD_LSN_COLS
+}
+
+/*
+ * Calculate size of a record, split into !FPI and FPI parts.
+ */
+static void
+GetXLogRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
+{
+	int			block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data. It doesn't seem worth it to
+	 * add an accessor macro for this.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += record->blocks[block_id].bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32		rec_len;
+	uint32		fpi_len;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int			block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	   *data;
+	char	   *main_data;
+	uint32		main_data_len;
+	int			i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(rec_len);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+
+	initStringInfo(&rec_desc);
+
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+
+	pfree(temp.data);
+
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		}
+		else
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+		}
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = PointerGetDatum(data);
+	values[i++] = UInt32GetDatum(main_data_len);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get the WAL record info and data.
+ *
+ * XXX: Currently, this function will not wait if the future WAL LSN is
+ * provided. We can have a wait version of it by using
+ * ReadLocalXLOGPage2Private.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum		result;
+	HeapTuple	tuple;
+	Datum		values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record LSN")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record, true, false);
+
+	Assert(xlogreader);
+
+	if (XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	if (record == NULL &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+
+		PG_RETURN_NULL();
+	}
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+					  PG_GET_WAL_RECORD_INFO_COLS);
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn, bool wait_for_wal)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool		nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true,
+									 wait_for_wal);
+
+	Assert(xlogreader);
+
+	if (wait_for_wal == false &&
+		XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+		tuplestore_end(tupstore);
+
+		return;
+	}
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		XLogRecord *record;
+
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		/* Exit loop if end-of-WAL is reached. */
+		if (record == NULL &&
+			IsEndOfWALReached(xlogreader))
+			break;
+
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+						  PG_GET_WAL_RECORDS_INFO_COLS);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function will wait if the future WAL LSN is provided and wait_for_wal
+ * is true. By default, it doesn't wait for the future WAL LSN, but emits a
+ * warning and returns the available info.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	bool		wait_for_wal;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+	wait_for_wal = PG_GETARG_BOOL(2);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	if (XLogRecPtrIsInvalid(end_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record end LSN")));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL record start LSN must be less than end LSN")));
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, end_lsn, wait_for_wal);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get info and data of all WAL records from start LSN till end-of-WAL.
+ *
+ * This function is similar to pg_get_wal_records_info except that it doesn't
+ * have end LSN as input and it doesn't wait for future WAL.
+ */
+Datum
+pg_get_wal_records_info_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, InvalidXLogRecPtr,
+							  false);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+static void
+StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record)
+{
+	RmgrId		rmid;
+	uint8		recid;
+	uint32		rec_len;
+	uint32		fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics. */
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double		n_pct;
+	double		rec_len_pct;
+	double		fpi_len_pct;
+	double		tot_len_pct;
+	int			i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+					TupleDesc tupdesc, Datum *values, bool *nulls,
+					uint32 ncols)
+{
+	int			ri;
+	uint64		total_count = 0;
+	uint64		total_rec_len = 0;
+	uint64		total_fpi_len = 0;
+	uint64		total_len = 0;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls, ncols);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+					XLogRecPtr end_lsn, bool wait_for_wal)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogRecStats stats;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum		values[PG_GET_WAL_STATS_COLS];
+	bool		nulls[PG_GET_WAL_STATS_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record, true,
+									 wait_for_wal);
+
+	Assert(xlogreader);
+
+	if (wait_for_wal == false &&
+		XLogRecPtrIsInvalid(first_record) &&
+		IsEndOfWALReached(xlogreader))
+	{
+		pfree(xlogreader->private_data);
+		XLogReaderFree(xlogreader);
+		tuplestore_end(tupstore);
+
+		return;
+	}
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		XLogRecord *record;
+
+		record = ReadNextXLogRecord(xlogreader, first_record);
+
+		/* Exit loop if end-of-WAL is reached. */
+		if (record == NULL &&
+			IsEndOfWALReached(xlogreader))
+			break;
+
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->currRecPtr >= end_lsn)
+			break;
+
+		StoreXLogRecordStats(&stats, xlogreader);
+	}
+
+	if (xlogreader->private_data)
+		pfree(xlogreader->private_data);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, tupstore, tupdesc, values, nulls,
+						PG_GET_WAL_STATS_COLS);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ *
+ * This function will wait if the future WAL LSN is provided and wait_for_wal
+ * is true. By default, it doesn't wait for the future WAL LSN, but emits a
+ * warning and returns the available info.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	bool		wait_for_wal;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+	wait_for_wal = PG_GETARG_BOOL(2);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	if (XLogRecPtrIsInvalid(end_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record end LSN")));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL record start LSN must be less than end LSN")));
+
+	GetWalStatsInternal(fcinfo, start_lsn, end_lsn, wait_for_wal);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get WAL stats from start LSN till end-of-WAL.
+ *
+ * This function is similar to pg_get_wal_stats except that it doesn't have end
+ * LSN as input and it doesn't wait for future WAL.
+ */
+Datum
+pg_get_wal_stats_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* Validate input. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL record start LSN")));
+
+	GetWalStatsInternal(fcinfo, start_lsn, InvalidXLogRecPtr, false);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index 35029cf97d..4d217cabbe 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -956,13 +956,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1080,6 +1073,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 54d5f20734..fd0ff07fbf 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -968,6 +968,150 @@ read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 	return count;
 }
 
+/*
+ * XLogReaderRoutine->page_read callback for reading local xlog files
+ *
+ * This function is same as read_local_xlog_page except that it works in both
+ * wait and no wait mode. The callers can specify about waiting in private_data
+ * of XLogReaderState.
+ */
+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+					   int reqLen, XLogRecPtr targetRecPtr, char *cur_page)
+{
+	XLogRecPtr	read_upto,
+				loc;
+	TimeLineID	tli;
+	int			count;
+	WALReadError errinfo;
+	TimeLineID	currTLI;
+
+	loc = targetPagePtr + reqLen;
+
+	/* Loop waiting for xlog to be available if necessary */
+	while (1)
+	{
+		/*
+		 * Determine the limit of xlog we can currently read to, and what the
+		 * most recent timeline is.
+		 */
+		if (!RecoveryInProgress())
+			read_upto = GetFlushRecPtr(&currTLI);
+		else
+			read_upto = GetXLogReplayRecPtr(&currTLI);
+		tli = currTLI;
+
+		/*
+		 * Check which timeline to get the record from.
+		 *
+		 * We have to do it each time through the loop because if we're in
+		 * recovery as a cascading standby, the current timeline might've
+		 * become historical. We can't rely on RecoveryInProgress() because in
+		 * a standby configuration like
+		 *
+		 * A => B => C
+		 *
+		 * if we're a logical decoding session on C, and B gets promoted, our
+		 * timeline will change while we remain in recovery.
+		 *
+		 * We can't just keep reading from the old timeline as the last WAL
+		 * archive in the timeline will get renamed to .partial by
+		 * StartupXLOG().
+		 *
+		 * If that happens after our caller determined the TLI but before
+		 * we actually read the xlog page, we might still try to read from the
+		 * old (now renamed) segment and fail. There's not much we can do
+		 * about this, but it can only happen when we're a leaf of a cascading
+		 * standby whose primary gets promoted while we're decoding, so a
+		 * one-off ERROR isn't too bad.
+		 */
+		XLogReadDetermineTimeline(state, targetPagePtr, reqLen, tli);
+
+		if (state->currTLI == currTLI)
+		{
+
+			if (loc <= read_upto)
+				break;
+
+			/*
+			 * We are trying to read future WAL. Let's not wait for WAL to be
+			 * available if indicated.
+			 */
+			if (state->private_data != NULL)
+			{
+				ReadLocalXLOGPage2Private *private;
+
+				private = (ReadLocalXLOGPage2Private *) state->private_data;
+
+				if (private->no_wait == true)
+				{
+					private->reached_end_of_wal = true;
+					break;
+				}
+			}
+
+			CHECK_FOR_INTERRUPTS();
+			pg_usleep(1000L);
+		}
+		else
+		{
+			/*
+			 * We're on a historical timeline, so limit reading to the switch
+			 * point where we moved to the next timeline.
+			 *
+			 * We don't need to GetFlushRecPtr or GetXLogReplayRecPtr. We know
+			 * about the new timeline, so we must've received past the end of
+			 * it.
+			 */
+			read_upto = state->currTLIValidUntil;
+
+			/*
+			 * Setting tli to our wanted record's TLI is slightly wrong; the
+			 * page might begin on an older timeline if it contains a timeline
+			 * switch, since its xlog segment will have been copied from the
+			 * prior timeline. This is pretty harmless though, as nothing
+			 * cares so long as the timeline doesn't go backwards.  We should
+			 * read the page header instead; FIXME someday.
+			 */
+			tli = state->currTLI;
+
+			/* No need to wait on a historical timeline */
+			break;
+		}
+	}
+
+	if (targetPagePtr + XLOG_BLCKSZ <= read_upto)
+	{
+		/*
+		 * more than one block available; read only that block, have caller
+		 * come back if they need more.
+		 */
+		count = XLOG_BLCKSZ;
+	}
+	else if (targetPagePtr + reqLen > read_upto)
+	{
+		/* not enough data there */
+		return -1;
+	}
+	else
+	{
+		/* enough bytes available to satisfy the request */
+		count = read_upto - targetPagePtr;
+	}
+
+	/*
+	 * Even though we just determined how much of the page can be validly read
+	 * as 'count', read the whole page anyway. It's guaranteed to be
+	 * zero-padded up to the page boundary if it's incomplete.
+	 */
+	if (!WALRead(state, cur_page, targetPagePtr, XLOG_BLCKSZ, tli,
+				 &errinfo))
+		WALReadRaiseError(&errinfo);
+
+	/* number of valid bytes in the buffer */
+	return count;
+}
+
 /*
  * Backend-specific convenience code to handle read errors encountered by
  * WALRead().
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 2340dc247b..60940d30c2 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 4b45ac64db..f34f228563 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 849954a8e5..38fd51bafa 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 477f0efe26..d62d6ce7f8 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
diff --git a/src/include/access/xlogutils.h b/src/include/access/xlogutils.h
index 64708949db..dd28640f07 100644
--- a/src/include/access/xlogutils.h
+++ b/src/include/access/xlogutils.h
@@ -75,6 +75,16 @@ typedef enum
 								 * need to be replayed) */
 } XLogRedoAction;
 
+/*
+ * private_data of read_local_xlog_page, that is, XLogReaderRoutine->page_read
+ * callback for reading local xlog files.
+ */
+typedef struct ReadLocalXLOGPage2Private
+{
+	bool no_wait;
+	bool reached_end_of_wal;
+} ReadLocalXLOGPage2Private;
+
 extern XLogRedoAction XLogReadBufferForRedo(XLogReaderState *record,
 											uint8 buffer_id, Buffer *buf);
 extern Buffer XLogInitBufferForRedo(XLogReaderState *record, uint8 block_id);
@@ -92,6 +102,9 @@ extern void FreeFakeRelcacheEntry(Relation fakerel);
 extern int	read_local_xlog_page(XLogReaderState *state,
 								 XLogRecPtr targetPagePtr, int reqLen,
 								 XLogRecPtr targetRecPtr, char *cur_page);
+extern int	read_local_xlog_page_2(XLogReaderState *state,
+								   XLogRecPtr targetPagePtr, int reqLen,
+								   XLogRecPtr targetRecPtr, char *cur_page);
 extern void wal_segment_open(XLogReaderState *state,
 							 XLogSegNo nextSegNo,
 							 TimeLineID *tli_p);
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v6-0001-pg_walinspect-tests.patchapplication/octet-stream; name=v6-0001-pg_walinspect-tests.patchDownload
From 361cf959bc8b5cffba5bbe065c785875e847cb23 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 2 Mar 2022 17:04:35 +0000
Subject: [PATCH v6] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 122 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   |  86 ++++++++++++
 2 files changed, 208 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..ba09c1caab
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,122 @@
+CREATE EXTENSION pg_walinspect;
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+CREATE TABLE sample_tbl(col1 int, col2 int);
+INSERT INTO sample_tbl VALUES(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl VALUES(2, 3);
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL record LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL record start LSN
+SELECT pg_get_wal_records_info(:'wal_lsn1', '0/0'); -- ERROR
+ERROR:  invalid WAL record end LSN
+SELECT pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+ERROR:  WAL record start LSN must be less than end LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL record start LSN
+SELECT pg_get_wal_stats(:'wal_lsn1', '0/0'); -- ERROR
+ERROR:  invalid WAL record end LSN
+SELECT pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+ERROR:  WAL record start LSN must be less than end LSN
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT lsn AS valid_wal_lsn1 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1') \gset
+SELECT lsn AS valid_wal_lsn2 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn2') \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'valid_wal_lsn2');
+WARNING:  reached end of WAL
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'valid_wal_lsn2');
+WARNING:  reached end of WAL
+ ok 
+----
+ t
+(1 row)
+
+SELECT file_name AS wal_file_name FROM pg_walfile_name_offset(:'valid_wal_lsn1') \gset
+SELECT COUNT(*) = 1 AS ok FROM pg_switch_wal();
+ ok 
+----
+ t
+(1 row)
+
+INSERT INTO sample_tbl VALUES(3, 4);
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_and_last_valid_wal_record_lsn(:'wal_file_name');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for functions with future WAL
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', 'FFFFFFFF/FFFFFFFF'); -- WARNING
+WARNING:  reached end of WAL
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', 'FFFFFFFF/FFFFFFFF'); -- WARNING
+WARNING:  reached end of WAL
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..5c4253787d
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,86 @@
+CREATE EXTENSION pg_walinspect;
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+INSERT INTO sample_tbl VALUES(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl VALUES(2, 3);
+
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info(:'wal_lsn1', '0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats(:'wal_lsn1', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- start LSN >= end LSN, ERROR
+
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+
+SELECT lsn AS valid_wal_lsn1 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1') \gset
+
+SELECT lsn AS valid_wal_lsn2 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn2') \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'valid_wal_lsn2');
+
+SELECT file_name AS wal_file_name FROM pg_walfile_name_offset(:'valid_wal_lsn1') \gset
+
+SELECT COUNT(*) = 1 AS ok FROM pg_switch_wal();
+
+INSERT INTO sample_tbl VALUES(3, 4);
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_and_last_valid_wal_record_lsn(:'wal_file_name');
+
+-- ===================================================================
+-- tests for functions with future WAL
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', 'FFFFFFFF/FFFFFFFF'); -- WARNING
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', 'FFFFFFFF/FFFFFFFF'); -- WARNING
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v6-0001-pg_walinspect-docs.patchapplication/octet-stream; name=v6-0001-pg_walinspect-docs.patchDownload
From a42b246856b3cf873bb78213ca924a148bc4e1c5 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 2 Mar 2022 17:05:13 +0000
Subject: [PATCH v6] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 165 +++++++++++++++++++++++++++++++++
 3 files changed, 167 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index be9711c6f2..19614a42e1 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -130,6 +130,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index 328cd1f378..a2e8fd4a08 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -146,6 +146,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..fa9f1d9b6c
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,165 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging and analytical
+  purposes.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_monitor</literal> role. Access may be granted to others using
+  <command>GRANT</command>.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>pg_get_raw_wal_record(in_lsn pg_lsn, lsn OUT pg_lsn, record OUT bytea)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. Issues a warning if the given
+      LSN wasn't a pointer to the start of a record and also wasn't a pointer
+      to the beginning of a WAL segment file. This function will not wait if
+      the future WAL LSN is provided, instead emits a warning and returns an
+      empty row.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_first_valid_wal_record_lsn(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets first and previous valid WAL record LSNs of the given LSN. Issues
+      a warning if the given LSN wasn't a pointer to the start of a record and
+      also wasn't a pointer to the beginning of a WAL segment file. This
+      function will not wait if the future WAL LSN is provided, instead emits
+      a warning and returns an empty row.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_first_and_last_valid_wal_record_lsn(wal_file_name text, first_valid_lsn OUT pg_lsn, last_valid_lsn OUT pg_lsn)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets first and last valid WAL record LSNs of the given WAL file.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_wal_record_info(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of the given LSN. Issues a warning if the
+      given LSN wasn't a pointer to the start of a record and also wasn't a
+      pointer to the beginning of a WAL segment file. This function will not
+      wait if the future WAL LSN is provided, instead emits a warning and
+      returns an empty row.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_records_info(start_lsn pg_lsn, end_lsn pg_lsn, wait_for_wal boolean DEFAULT false, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. Issues a warning if the given
+      <replaceable>start_lsn</replaceable> wasn't a pointer to the start of a
+      record and also wasn't a pointer to the beginning of a WAL segment file.
+      This function will wait if the future WAL LSN is provided when
+      <replaceable>wait_for_wal</replaceable> is passed as <literal>true</literal>.
+      If <replaceable>wait_for_wal</replaceable> is <literal>false</literal>
+      (default value), it will not wait for the future WAL, instead emits a
+      warning and returns rows for whatever available WAL records.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_records_info_till_end_of_wal(start_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL. This function is
+      similar to <literal>pg_get_wal_records_info</literal> except that it
+      doesn't have <replaceable>end_lsn</replaceable> as input and it doesn't
+      wait for the future WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_stats(start_lsn pg_lsn, end_lsn pg_lsn, wait_for_wal boolean DEFAULT false, resource_manager OUT text, count OUT int8, count_percentage OUT float4, record_size OUT int8, record_size_percentage OUT float4, fpi_size OUT int8, fpi_size_percentage OUT float4, combined_size OUT int8, combined_size_percentage OUT float4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each <replaceable>resource_manager</replaceable>
+      type. Issues a warning if the given <replaceable>start_lsn</replaceable>
+      wasn't a pointer to the start of a record and also wasn't a pointer to
+      the beginning of a WAL segment file. This function will wait if the
+      future WAL LSN is provided when <replaceable>wait_for_wal</replaceable>
+      is passed as <literal>true</literal>. If <replaceable>wait_for_wal</replaceable>
+      is <literal>false</literal> (default value), it will not wait for the
+      future WAL, instead emits a warning and returns rows for whatever
+      available WAL records.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_stats_till_end_of_wal(start_lsn pg_lsn, end_lsn pg_lsn, resource_manager OUT text, count OUT int8, count_percentage OUT float4, record_size OUT int8, record_size_percentage OUT float4, fpi_size OUT int8, fpi_size_percentage OUT float4, combined_size OUT int8, combined_size_percentage OUT float4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL. This function is
+      similar to <literal>pg_get_wal_stats</literal> except that it doesn't
+      have <replaceable>end_lsn</replaceable> as input and it doesn't wait for
+      the future WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#36Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Bharath Rupireddy (#35)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Mar 2, 2022 at 10:37 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

On Wed, Mar 2, 2022 at 8:12 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

Some review comments on v5 patch (v5-0001-pg_walinspect.patch)

Thanks for reviewing.

+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT lsn pg_lsn,

What does the wait_for_wal flag mean here when one has already
specified the start and end lsn? AFAIU, If a user has specified a
start and stop LSN, it means that the user knows the extent to which
he/she wants to display the WAL records in which case we need to stop
once the end lsn has reached . So what is the meaning of wait_for_wal
flag? Does it look sensible to have the wait_for_wal flag here? To me
it doesn't.

Users can always specify a future end_lsn and set wait_for_wal to
true, then the pg_get_wal_records_info/pg_get_wal_stats functions can
wait for the WAL. IMO, this is useful. If you remember you were okay
with wait/nowait versions for some of the functions upthread [1]. I'm
not going to retain this behaviour for both
pg_get_wal_records_info/pg_get_wal_stats as it is similar to
pg_waldump's --follow option.

It is not at all similar to pg_waldumps behaviour. Please check the
behaviour of pg_waldump properly. Does it wait for any wal records
when a user has specified a stop pointer? It doesn't and it shouldn't.
I mean does it even make sense to wait for the WAL when a stop pointer
is specified? And it's quite understandable that if a user has asked
pg_walinspect to stop at a certain point, it must. Also, What if there
are already WAL records after the stop pointer, in that case does it
even make sense to have a wait flag. WHat would be the meaning of the
wait flag in that case?

Further, have you checked wait_for_wal flag behaviour, is it even working?

+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,

Why is this function required? Is pg_get_wal_records_info() alone not
enough? I think it is. See if we can make end_lsn optional in
pg_get_wal_records_info() and lets just have it alone. I think it can
do the job of pg_get_wal_records_info_till_end_of_wal function.

==

+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,

Above comment applies to this function as well. Isn't pg_get_wal_stats() enough?

I'm doing the following input validations for these functions to not
cause any issues with invalid LSN. If I were to have the default value
for end_lsn as 0/0, I can't perform input validations right? That is
the reason I'm having separate functions {pg_get_wal_records_info,
pg_get_wal_stats}_till_end_of_wal() versions.

You can do it. Please check pg_waldump to understand how it is done
there. You cannot have multiple functions doing different things when
one single function can do all the job.

Show quoted text

==

+                       if (loc <= read_upto)
+                               break;
+
+                       /* Let's not wait for WAL to be available if
indicated */
+                       if (loc > read_upto &&
+                               state->private_data != NULL)
+                       {

Why loc > read_upto? The first if condition is (loc <= read_upto)
followed by the second if condition - (loc > read_upto). Is the first
if condition (loc <= read_upto) not enough to indicate that loc >
read_upto?

Yeah, that's unnecessary, I improved the comment there and removed loc

read_upto.

==

+#define IsEndOfWALReached(state) \
+               (state->private_data != NULL && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->no_wait == true) && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))

I think we should either use state or xlogreader. First line says
state->private_data and second line xlogreader->private_data.

I've changed it to use state instead of xlogreader.

==

+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))
+

There is a new patch coming to make the end of WAL messages less
scary. It introduces the EOW flag in xlogreaderstate maybe we can use
that instead of introducing new flags in private area to represent the
end of WAL.

Yeah that would be great. But we never know which one gets committed
first. Until then it's not good to have dependencies on two "on-going"
patches. Later, we can change.

==

+/*
+ * XLogReaderRoutine->page_read callback for reading local xlog files
+ *
+ * This function is same as read_local_xlog_page except that it works in both
+ * wait and no wait mode. The callers can specify about waiting in private_data
+ * of XLogReaderState.
+ */
+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+                                          int reqLen, XLogRecPtr
targetRecPtr, char *cur_page)
+{
+       XLogRecPtr      read_upto,

Do we really need this function? Can't we make use of an existing WAL
reader function - read_local_xlog_page()?

I clearly explained the reasons upthread [2]. Please let me know if
you have more thoughts/doubts here, we can connect offlist.

Attaching v6 patch set with above review comments addressed. Please
review it further.

[1] https://www.postgresql.org/message-id/CAE9k0P%3D9SReU_613TXytZmpwL3ZRpnC5zrf96UoNCATKpK-UxQ%40mail.gmail.com
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

[2] /messages/by-id/CALj2ACUtqWX95uAj2VNJED0PnixEeQ=0MEzpouLi+zd_iTugRA@mail.gmail.com
I've added a new function read_local_xlog_page_2 (similar to
read_local_xlog_page but works in wait and no wait mode) and the
callers can specify whether to wait or not wait using private_data.
Actually, I wanted to use the private_data structure of
read_local_xlog_page but the logical decoding already has context as
private_data, that is why I had to have a new function. I know it
creates a bit of duplicate code, but its cleaner than using
backend-local variables or additional flags in XLogReaderState or
adding wait/no-wait boolean to page_read callback. Any other
suggestions are welcome here.

With this, I'm able to have wait/no wait versions for any functions.
But for now, I'm having wait/no wait for two functions
(pg_get_wal_records_info and pg_get_wal_stats) for which it makes more
sense.

Regards,
Bharath Rupireddy.

#37Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Bharath Rupireddy (#35)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Hi.

+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */

Why didn't you remove the emptied #ifdef section?

+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+					   int reqLen, XLogRecPtr targetRecPtr, char *cur_page)

The difference with the original function is this function has one
additional if-block amid. I think we can insert the code directly in
the original function.

+			/*
+			 * We are trying to read future WAL. Let's not wait for WAL to be
+			 * available if indicated.
+			 */
+			if (state->private_data != NULL)

However, in the first place it seems to me there's not need for the
function to take care of no_wait affairs.

If, for expample, pg_get_wal_record_info() with no_wait = true, it is
enough that the function identifies the bleeding edge of WAL then loop
until the LSN. So I think no need for the new function, nor for any
modification on the origical function.

The changes will reduce the footprint of the patch largely, I think.

At Wed, 2 Mar 2022 22:37:43 +0530, Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com> wrote in

On Wed, Mar 2, 2022 at 8:12 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

Some review comments on v5 patch (v5-0001-pg_walinspect.patch)

Thanks for reviewing.

+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT lsn pg_lsn,

What does the wait_for_wal flag mean here when one has already
specified the start and end lsn? AFAIU, If a user has specified a
start and stop LSN, it means that the user knows the extent to which
he/she wants to display the WAL records in which case we need to stop
once the end lsn has reached . So what is the meaning of wait_for_wal
flag? Does it look sensible to have the wait_for_wal flag here? To me
it doesn't.

Users can always specify a future end_lsn and set wait_for_wal to
true, then the pg_get_wal_records_info/pg_get_wal_stats functions can
wait for the WAL. IMO, this is useful. If you remember you were okay
with wait/nowait versions for some of the functions upthread [1]. I'm
not going to retain this behaviour for both
pg_get_wal_records_info/pg_get_wal_stats as it is similar to
pg_waldump's --follow option.

I agree to this for now. However, I prefer that NULL or invalid
end_lsn is equivalent to pg_current_wal_lsn().

==

+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,

Why is this function required? Is pg_get_wal_records_info() alone not
enough? I think it is. See if we can make end_lsn optional in
pg_get_wal_records_info() and lets just have it alone. I think it can
do the job of pg_get_wal_records_info_till_end_of_wal function.

I rather agree to Ashutosh. This feature can be covered by
pg_get_wal_records(start_lsn, NULL, false).

==

+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,

Above comment applies to this function as well. Isn't pg_get_wal_stats() enough?

I'm doing the following input validations for these functions to not
cause any issues with invalid LSN. If I were to have the default value
for end_lsn as 0/0, I can't perform input validations right? That is
the reason I'm having separate functions {pg_get_wal_records_info,
pg_get_wal_stats}_till_end_of_wal() versions.

/* Validate input. */
if (XLogRecPtrIsInvalid(start_lsn))
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("invalid WAL record start LSN")));

if (XLogRecPtrIsInvalid(end_lsn))
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("invalid WAL record end LSN")));

if (start_lsn >= end_lsn)
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("WAL record start LSN must be less than end LSN")));

I don't think that validations are worth doing at least for the first
two, as far as that value has a special meaning. I see it useful if
pg_get_wal_records_info() means dump the all available records for the
moment, or records of the last segment, page or something.

==

+                       if (loc <= read_upto)
+                               break;
+
+                       /* Let's not wait for WAL to be available if
indicated */
+                       if (loc > read_upto &&
+                               state->private_data != NULL)
+                       {

Why loc > read_upto? The first if condition is (loc <= read_upto)
followed by the second if condition - (loc > read_upto). Is the first
if condition (loc <= read_upto) not enough to indicate that loc >
read_upto?

Yeah, that's unnecessary, I improved the comment there and removed loc

read_upto.

==

+#define IsEndOfWALReached(state) \
+               (state->private_data != NULL && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->no_wait == true) && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))

I think we should either use state or xlogreader. First line says
state->private_data and second line xlogreader->private_data.

I've changed it to use state instead of xlogreader.

==

+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))
+

There is a new patch coming to make the end of WAL messages less
scary. It introduces the EOW flag in xlogreaderstate maybe we can use
that instead of introducing new flags in private area to represent the
end of WAL.

Yeah that would be great. But we never know which one gets committed
first. Until then it's not good to have dependencies on two "on-going"
patches. Later, we can change.

==

+/*
+ * XLogReaderRoutine->page_read callback for reading local xlog files
+ *
+ * This function is same as read_local_xlog_page except that it works in both
+ * wait and no wait mode. The callers can specify about waiting in private_data
+ * of XLogReaderState.
+ */
+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+                                          int reqLen, XLogRecPtr
targetRecPtr, char *cur_page)
+{
+       XLogRecPtr      read_upto,

Do we really need this function? Can't we make use of an existing WAL
reader function - read_local_xlog_page()?

I clearly explained the reasons upthread [2]. Please let me know if
you have more thoughts/doubts here, we can connect offlist.

*I* also think the function is not needed, as explained above. Why do
we need that function while we know how far we can read WAL records
*before* calling the function?

Attaching v6 patch set with above review comments addressed. Please
review it further.

[1] https://www.postgresql.org/message-id/CAE9k0P%3D9SReU_613TXytZmpwL3ZRpnC5zrf96UoNCATKpK-UxQ%40mail.gmail.com
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

[2] /messages/by-id/CALj2ACUtqWX95uAj2VNJED0PnixEeQ=0MEzpouLi+zd_iTugRA@mail.gmail.com
I've added a new function read_local_xlog_page_2 (similar to
read_local_xlog_page but works in wait and no wait mode) and the
callers can specify whether to wait or not wait using private_data.
Actually, I wanted to use the private_data structure of
read_local_xlog_page but the logical decoding already has context as
private_data, that is why I had to have a new function. I know it
creates a bit of duplicate code, but its cleaner than using
backend-local variables or additional flags in XLogReaderState or
adding wait/no-wait boolean to page_read callback. Any other
suggestions are welcome here.

With this, I'm able to have wait/no wait versions for any functions.
But for now, I'm having wait/no wait for two functions
(pg_get_wal_records_info and pg_get_wal_stats) for which it makes more
sense.

Regards,
Bharath Rupireddy.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#38Nitin Jadhav
nitinjadhavpostgres@gmail.com
In reply to: Kyotaro Horiguchi (#37)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Here are a few comments.

1)

==

+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,

Why is this function required? Is pg_get_wal_records_info() alone not
enough? I think it is. See if we can make end_lsn optional in
pg_get_wal_records_info() and lets just have it alone. I think it can
do the job of pg_get_wal_records_info_till_end_of_wal function.

I rather agree to Ashutosh. This feature can be covered by
pg_get_wal_records(start_lsn, NULL, false).
I don't think that validations are worth doing at least for the first
two, as far as that value has a special meaning. I see it useful if
pg_get_wal_records_info() means dump the all available records for the
moment, or records of the last segment, page or something.
*I* also think the function is not needed, as explained above. Why do
we need that function while we know how far we can read WAL records
*before* calling the function?

I agree with this. The function prototype comes first and the
validation can be done accordingly. I feel we can even support
'pg_get_wal_record_info' with the same name. All 3 function's
objectives are the same. So it is better to use the same name
(pg_wal_record_info) with different prototypes.

2) The function 'pg_get_first_valid_wal_record_lsn' looks redundant as
we are getting the same information from the function
'pg_get_first_and_last_valid_wal_record_lsn'. With this function, we
can easily fetch the first lsn. So IMO we should remove
'pg_get_first_valid_wal_record_lsn'.

3) The word 'get' should be removed from the function name(*_get_*) as
all the functions of the extension are used only to get the
information. It will also sync with xlogfuncs's naming conventions
like pg_current_wal_lsn, pg_walfile_name, etc.

4) The function names can be modified with lesser words by retaining
the existing meaning.
:s/pg_get_raw_wal_record/pg_wal_raw_record
:s/pg_get_first_valid_wal_record_lsn/pg_wal_first_lsn
:s/pg_get_first_and_last_valid_wal_record_lsn/pg_wal_first_and_last_lsn
:s/pg_get_wal_record_info/pg_wal_record_info
:s/pg_get_wal_stats/pg_wal_stats

5) Even 'pg_get_wal_stats' and 'pg_get_wal_stats_till_end_of_wal' can
be clubbed as one function.

The above comments are trying to simplify the extension APIs and to
make it easy for the user to understand and use it.

Thanks & Regards,
Nitin Jadhav

On Thu, Mar 3, 2022 at 8:20 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Show quoted text

Hi.

+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
#endif                                                 /* FRONTEND */

Why didn't you remove the emptied #ifdef section?

+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+                                          int reqLen, XLogRecPtr targetRecPtr, char *cur_page)

The difference with the original function is this function has one
additional if-block amid. I think we can insert the code directly in
the original function.

+                       /*
+                        * We are trying to read future WAL. Let's not wait for WAL to be
+                        * available if indicated.
+                        */
+                       if (state->private_data != NULL)

However, in the first place it seems to me there's not need for the
function to take care of no_wait affairs.

If, for expample, pg_get_wal_record_info() with no_wait = true, it is
enough that the function identifies the bleeding edge of WAL then loop
until the LSN. So I think no need for the new function, nor for any
modification on the origical function.

The changes will reduce the footprint of the patch largely, I think.

At Wed, 2 Mar 2022 22:37:43 +0530, Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com> wrote in

On Wed, Mar 2, 2022 at 8:12 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

Some review comments on v5 patch (v5-0001-pg_walinspect.patch)

Thanks for reviewing.

+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT lsn pg_lsn,

What does the wait_for_wal flag mean here when one has already
specified the start and end lsn? AFAIU, If a user has specified a
start and stop LSN, it means that the user knows the extent to which
he/she wants to display the WAL records in which case we need to stop
once the end lsn has reached . So what is the meaning of wait_for_wal
flag? Does it look sensible to have the wait_for_wal flag here? To me
it doesn't.

Users can always specify a future end_lsn and set wait_for_wal to
true, then the pg_get_wal_records_info/pg_get_wal_stats functions can
wait for the WAL. IMO, this is useful. If you remember you were okay
with wait/nowait versions for some of the functions upthread [1]. I'm
not going to retain this behaviour for both
pg_get_wal_records_info/pg_get_wal_stats as it is similar to
pg_waldump's --follow option.

I agree to this for now. However, I prefer that NULL or invalid
end_lsn is equivalent to pg_current_wal_lsn().

==

+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,

Why is this function required? Is pg_get_wal_records_info() alone not
enough? I think it is. See if we can make end_lsn optional in
pg_get_wal_records_info() and lets just have it alone. I think it can
do the job of pg_get_wal_records_info_till_end_of_wal function.

I rather agree to Ashutosh. This feature can be covered by
pg_get_wal_records(start_lsn, NULL, false).

==

+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,

Above comment applies to this function as well. Isn't pg_get_wal_stats() enough?

I'm doing the following input validations for these functions to not
cause any issues with invalid LSN. If I were to have the default value
for end_lsn as 0/0, I can't perform input validations right? That is
the reason I'm having separate functions {pg_get_wal_records_info,
pg_get_wal_stats}_till_end_of_wal() versions.

/* Validate input. */
if (XLogRecPtrIsInvalid(start_lsn))
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("invalid WAL record start LSN")));

if (XLogRecPtrIsInvalid(end_lsn))
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("invalid WAL record end LSN")));

if (start_lsn >= end_lsn)
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("WAL record start LSN must be less than end LSN")));

I don't think that validations are worth doing at least for the first
two, as far as that value has a special meaning. I see it useful if
pg_get_wal_records_info() means dump the all available records for the
moment, or records of the last segment, page or something.

==

+                       if (loc <= read_upto)
+                               break;
+
+                       /* Let's not wait for WAL to be available if
indicated */
+                       if (loc > read_upto &&
+                               state->private_data != NULL)
+                       {

Why loc > read_upto? The first if condition is (loc <= read_upto)
followed by the second if condition - (loc > read_upto). Is the first
if condition (loc <= read_upto) not enough to indicate that loc >
read_upto?

Yeah, that's unnecessary, I improved the comment there and removed loc

read_upto.

==

+#define IsEndOfWALReached(state) \
+               (state->private_data != NULL && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->no_wait == true) && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))

I think we should either use state or xlogreader. First line says
state->private_data and second line xlogreader->private_data.

I've changed it to use state instead of xlogreader.

==

+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))
+

There is a new patch coming to make the end of WAL messages less
scary. It introduces the EOW flag in xlogreaderstate maybe we can use
that instead of introducing new flags in private area to represent the
end of WAL.

Yeah that would be great. But we never know which one gets committed
first. Until then it's not good to have dependencies on two "on-going"
patches. Later, we can change.

==

+/*
+ * XLogReaderRoutine->page_read callback for reading local xlog files
+ *
+ * This function is same as read_local_xlog_page except that it works in both
+ * wait and no wait mode. The callers can specify about waiting in private_data
+ * of XLogReaderState.
+ */
+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+                                          int reqLen, XLogRecPtr
targetRecPtr, char *cur_page)
+{
+       XLogRecPtr      read_upto,

Do we really need this function? Can't we make use of an existing WAL
reader function - read_local_xlog_page()?

I clearly explained the reasons upthread [2]. Please let me know if
you have more thoughts/doubts here, we can connect offlist.

*I* also think the function is not needed, as explained above. Why do
we need that function while we know how far we can read WAL records
*before* calling the function?

Attaching v6 patch set with above review comments addressed. Please
review it further.

[1] https://www.postgresql.org/message-id/CAE9k0P%3D9SReU_613TXytZmpwL3ZRpnC5zrf96UoNCATKpK-UxQ%40mail.gmail.com
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

[2] /messages/by-id/CALj2ACUtqWX95uAj2VNJED0PnixEeQ=0MEzpouLi+zd_iTugRA@mail.gmail.com
I've added a new function read_local_xlog_page_2 (similar to
read_local_xlog_page but works in wait and no wait mode) and the
callers can specify whether to wait or not wait using private_data.
Actually, I wanted to use the private_data structure of
read_local_xlog_page but the logical decoding already has context as
private_data, that is why I had to have a new function. I know it
creates a bit of duplicate code, but its cleaner than using
backend-local variables or additional flags in XLogReaderState or
adding wait/no-wait boolean to page_read callback. Any other
suggestions are welcome here.

With this, I'm able to have wait/no wait versions for any functions.
But for now, I'm having wait/no wait for two functions
(pg_get_wal_records_info and pg_get_wal_stats) for which it makes more
sense.

Regards,
Bharath Rupireddy.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#39Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Kyotaro Horiguchi (#37)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

I think we should also see if we can allow end users to input timeline
information with the pg_walinspect functions. This will help the end
users to get information about WAL records from previous timeline
which can be helpful in case of restored servers.

--
With Regards,
Ashutosh Sharma.

On Thu, Mar 3, 2022 at 8:20 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Show quoted text

Hi.

+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
#endif                                                 /* FRONTEND */

Why didn't you remove the emptied #ifdef section?

+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+                                          int reqLen, XLogRecPtr targetRecPtr, char *cur_page)

The difference with the original function is this function has one
additional if-block amid. I think we can insert the code directly in
the original function.

+                       /*
+                        * We are trying to read future WAL. Let's not wait for WAL to be
+                        * available if indicated.
+                        */
+                       if (state->private_data != NULL)

However, in the first place it seems to me there's not need for the
function to take care of no_wait affairs.

If, for expample, pg_get_wal_record_info() with no_wait = true, it is
enough that the function identifies the bleeding edge of WAL then loop
until the LSN. So I think no need for the new function, nor for any
modification on the origical function.

The changes will reduce the footprint of the patch largely, I think.

At Wed, 2 Mar 2022 22:37:43 +0530, Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com> wrote in

On Wed, Mar 2, 2022 at 8:12 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

Some review comments on v5 patch (v5-0001-pg_walinspect.patch)

Thanks for reviewing.

+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN wait_for_wal boolean DEFAULT false,
+    OUT lsn pg_lsn,

What does the wait_for_wal flag mean here when one has already
specified the start and end lsn? AFAIU, If a user has specified a
start and stop LSN, it means that the user knows the extent to which
he/she wants to display the WAL records in which case we need to stop
once the end lsn has reached . So what is the meaning of wait_for_wal
flag? Does it look sensible to have the wait_for_wal flag here? To me
it doesn't.

Users can always specify a future end_lsn and set wait_for_wal to
true, then the pg_get_wal_records_info/pg_get_wal_stats functions can
wait for the WAL. IMO, this is useful. If you remember you were okay
with wait/nowait versions for some of the functions upthread [1]. I'm
not going to retain this behaviour for both
pg_get_wal_records_info/pg_get_wal_stats as it is similar to
pg_waldump's --follow option.

I agree to this for now. However, I prefer that NULL or invalid
end_lsn is equivalent to pg_current_wal_lsn().

==

+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,

Why is this function required? Is pg_get_wal_records_info() alone not
enough? I think it is. See if we can make end_lsn optional in
pg_get_wal_records_info() and lets just have it alone. I think it can
do the job of pg_get_wal_records_info_till_end_of_wal function.

I rather agree to Ashutosh. This feature can be covered by
pg_get_wal_records(start_lsn, NULL, false).

==

+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,

Above comment applies to this function as well. Isn't pg_get_wal_stats() enough?

I'm doing the following input validations for these functions to not
cause any issues with invalid LSN. If I were to have the default value
for end_lsn as 0/0, I can't perform input validations right? That is
the reason I'm having separate functions {pg_get_wal_records_info,
pg_get_wal_stats}_till_end_of_wal() versions.

/* Validate input. */
if (XLogRecPtrIsInvalid(start_lsn))
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("invalid WAL record start LSN")));

if (XLogRecPtrIsInvalid(end_lsn))
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("invalid WAL record end LSN")));

if (start_lsn >= end_lsn)
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("WAL record start LSN must be less than end LSN")));

I don't think that validations are worth doing at least for the first
two, as far as that value has a special meaning. I see it useful if
pg_get_wal_records_info() means dump the all available records for the
moment, or records of the last segment, page or something.

==

+                       if (loc <= read_upto)
+                               break;
+
+                       /* Let's not wait for WAL to be available if
indicated */
+                       if (loc > read_upto &&
+                               state->private_data != NULL)
+                       {

Why loc > read_upto? The first if condition is (loc <= read_upto)
followed by the second if condition - (loc > read_upto). Is the first
if condition (loc <= read_upto) not enough to indicate that loc >
read_upto?

Yeah, that's unnecessary, I improved the comment there and removed loc

read_upto.

==

+#define IsEndOfWALReached(state) \
+               (state->private_data != NULL && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->no_wait == true) && \
+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))

I think we should either use state or xlogreader. First line says
state->private_data and second line xlogreader->private_data.

I've changed it to use state instead of xlogreader.

==

+               (((ReadLocalXLOGPage2Private *)
xlogreader->private_data)->reached_end_of_wal == true))
+

There is a new patch coming to make the end of WAL messages less
scary. It introduces the EOW flag in xlogreaderstate maybe we can use
that instead of introducing new flags in private area to represent the
end of WAL.

Yeah that would be great. But we never know which one gets committed
first. Until then it's not good to have dependencies on two "on-going"
patches. Later, we can change.

==

+/*
+ * XLogReaderRoutine->page_read callback for reading local xlog files
+ *
+ * This function is same as read_local_xlog_page except that it works in both
+ * wait and no wait mode. The callers can specify about waiting in private_data
+ * of XLogReaderState.
+ */
+int
+read_local_xlog_page_2(XLogReaderState *state, XLogRecPtr targetPagePtr,
+                                          int reqLen, XLogRecPtr
targetRecPtr, char *cur_page)
+{
+       XLogRecPtr      read_upto,

Do we really need this function? Can't we make use of an existing WAL
reader function - read_local_xlog_page()?

I clearly explained the reasons upthread [2]. Please let me know if
you have more thoughts/doubts here, we can connect offlist.

*I* also think the function is not needed, as explained above. Why do
we need that function while we know how far we can read WAL records
*before* calling the function?

Attaching v6 patch set with above review comments addressed. Please
review it further.

[1] https://www.postgresql.org/message-id/CAE9k0P%3D9SReU_613TXytZmpwL3ZRpnC5zrf96UoNCATKpK-UxQ%40mail.gmail.com
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_verify_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);

I think we should allow all these functions to be executed in wait and
*nowait* mode. If a user specifies nowait mode, the function should
return if no WAL data is present, rather than waiting for new WAL data
to become available, default behaviour could be anything you like.

[2] /messages/by-id/CALj2ACUtqWX95uAj2VNJED0PnixEeQ=0MEzpouLi+zd_iTugRA@mail.gmail.com
I've added a new function read_local_xlog_page_2 (similar to
read_local_xlog_page but works in wait and no wait mode) and the
callers can specify whether to wait or not wait using private_data.
Actually, I wanted to use the private_data structure of
read_local_xlog_page but the logical decoding already has context as
private_data, that is why I had to have a new function. I know it
creates a bit of duplicate code, but its cleaner than using
backend-local variables or additional flags in XLogReaderState or
adding wait/no-wait boolean to page_read callback. Any other
suggestions are welcome here.

With this, I'm able to have wait/no wait versions for any functions.
But for now, I'm having wait/no wait for two functions
(pg_get_wal_records_info and pg_get_wal_stats) for which it makes more
sense.

Regards,
Bharath Rupireddy.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#40Robert Haas
robertmhaas@gmail.com
In reply to: Bharath Rupireddy (#33)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Feb 25, 2022 at 6:03 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Added a new function that returns the first and last valid WAL record
LSN of a given WAL file.

Sounds like fuzzy thinking to me. WAL records can cross file
boundaries, and forgetting about that leads to all sorts of problems.
Just give people one function that decodes a range of LSNs and call it
good. Why do you need anything else? If people want to get the first
record that begins in a segment or the first record any portion of
which is in a particular segment or the last record that begins in a
segment or the last record that ends in a segment or any other such
thing, they can use a WHERE clause for that... and if you think they
can't, then that should be good cause to rethink the return value of
the one-and-only SRF that I think you need here.

--
Robert Haas
EDB: http://www.enterprisedb.com

#41Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Robert Haas (#40)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Mar 3, 2022 at 10:05 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Feb 25, 2022 at 6:03 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Added a new function that returns the first and last valid WAL record
LSN of a given WAL file.

Sounds like fuzzy thinking to me. WAL records can cross file
boundaries, and forgetting about that leads to all sorts of problems.
Just give people one function that decodes a range of LSNs and call it
good. Why do you need anything else? If people want to get the first
record that begins in a segment or the first record any portion of
which is in a particular segment or the last record that begins in a
segment or the last record that ends in a segment or any other such
thing, they can use a WHERE clause for that... and if you think they
can't, then that should be good cause to rethink the return value of
the one-and-only SRF that I think you need here.

Thanks Robert.

Thanks to others for your review comments.

Here's the v7 patch set. These patches are based on the motive that
"keep it simple and short yet effective and useful". With that in
mind, I have not implemented the wait mode for any of the functions
(as it doesn't look an effective use-case and requires adding a new
page_read callback, instead throw error if future LSN is specified),
also these functions will give WAL information on the current server's
timeline. Having said that, I'm open to adding new functions in future
once this initial version gets in, if there's a requirement and users
ask for the new functions.

Please review the v7 patch set and provide your thoughts.

Regards,
Bharath Rupireddy.

Attachments:

v7-0001-pg_walinspect-tests.patchapplication/x-patch; name=v7-0001-pg_walinspect-tests.patchDownload
From fc6c124efa87ff3c9840bcf444697f762a4cf1fe Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 4 Mar 2022 08:29:35 +0000
Subject: [PATCH v7] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 69 +++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 58 ++++++++++++++++
 2 files changed, 127 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..ad295976dd
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,69 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT lsn AS valid_wal_lsn1 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1') \gset
+SELECT lsn AS valid_wal_lsn2 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn2') \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..013b285bdb
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,58 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+
+SELECT lsn AS valid_wal_lsn1 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1') \gset
+
+SELECT lsn AS valid_wal_lsn2 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn2') \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v7-0001-pg_walinspect-docs.patchapplication/x-patch; name=v7-0001-pg_walinspect-docs.patchDownload
From 1efcb8fc07cf5f5b1b2987339321dd803c96275a Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 4 Mar 2022 09:15:21 +0000
Subject: [PATCH v7] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 128 +++++++++++++++++++++++++++++++++
 3 files changed, 130 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index be9711c6f2..19614a42e1 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -130,6 +130,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index 328cd1f378..a2e8fd4a08 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -146,6 +146,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..5bc79c80ae
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,128 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_monitor</literal> role. Access may be granted to others using
+  <command>GRANT</command>.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>pg_get_raw_wal_record(in_lsn pg_lsn, lsn OUT pg_lsn, record OUT bytea)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. Issues a warning if the given
+      LSN wasn't a pointer to the start of a record and also wasn't a pointer
+      to the beginning of a WAL segment file. This function emits an error if
+      a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_first_valid_wal_record_lsn(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets first and previous valid WAL record LSNs of the given LSN. Issues
+      a warning if the given LSN wasn't a pointer to the start of a record and
+      also wasn't a pointer to the beginning of a WAL segment file. This
+      function emits an error if a future (the LSN database system doesn't know
+      about) <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_wal_record_info(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of the given LSN. Issues a warning if the
+      given LSN wasn't a pointer to the start of a record and also wasn't a
+      pointer to the beginning of a WAL segment file. This function emits an
+      error if a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_records_info(start_lsn pg_lsn, end_lsn pg_lsn DEFAULT NULL, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. Issues a warning if the given
+      <replaceable>start_lsn</replaceable> wasn't a pointer to the start of a
+      record and also wasn't a pointer to the beginning of a WAL segment file.
+      This function figures out the <replaceable>end_lsn</replaceable> if it's
+      not specified, that means, it returns information up to the end of WAL.
+      Default value of <replaceable>end_lsn</replaceable> is <literal>NULL</literal>.
+      This function emits an error if a future (the LSN database system doesn't
+      know about) <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_stats(start_lsn pg_lsn, end_lsn pg_lsn, wait_for_wal boolean DEFAULT false, resource_manager OUT text, count OUT int8, count_percentage OUT float4, record_size OUT int8, record_size_percentage OUT float4, fpi_size OUT int8, fpi_size_percentage OUT float4, combined_size OUT int8, combined_size_percentage OUT float4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each <replaceable>resource_manager</replaceable>
+      type. Issues a warning if the given <replaceable>start_lsn</replaceable>
+      wasn't a pointer to the start of a record and also wasn't a pointer to
+      the beginning of a WAL segment file. This function figures out the
+      <replaceable>end_lsn</replaceable> if it's not specified, that means, it
+      returns information up to the end of WAL. Default value of
+      <replaceable>end_lsn</replaceable> is <literal>NULL</literal>. This
+      function emits an error if a future (the LSN database system doesn't know
+      about) <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+</sect1>
-- 
2.25.1

v7-0001-pg_walinspect.patchapplication/x-patch; name=v7-0001-pg_walinspect.patchDownload
From df9e5b5f9a76a2002d9ef8d9b0db88003a07a5b5 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 4 Mar 2022 08:12:20 +0000
Subject: [PATCH v7] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql |  91 ++
 contrib/pg_walinspect/pg_walinspect.c        | 949 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  14 +-
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/common/relpath.c                         |  18 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/common/relpath.h                 |   1 +
 13 files changed, 1109 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index e3e221308b..705c6fc36b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -40,6 +40,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..619b1d1d4a
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,91 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_first_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_valid_wal_record_lsn(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_valid_wal_record_lsn'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..99f2f43c42
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,949 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct Stats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+}	Stats;
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	Stats		rmgr_stats[RM_NEXT_ID];
+	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+}	XLogRecStats;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+
+static XLogRecPtr ValidateInputLSN(XLogRecPtr lsn);
+static XLogRecPtr ValidateStartAndEndLSNs(XLogRecPtr start_lsn,
+										  XLogRecPtr end_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void StoreXLogRecordStats(XLogRecStats * stats,
+								 XLogReaderState *record);
+static void GetXLogSummaryStats(XLogRecStats * stats,
+								Tuplestorestate *tupstore, TupleDesc tupdesc,
+								Datum *values, bool *nulls, uint32 ncols);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+								XLogRecPtr end_lsn);
+static void GetWALRecordsInfoInternal(FunctionCallInfo fcinfo,
+									  XLogRecPtr start_lsn,
+									  XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Validate given LSN and return the LSN up to which the server has WAL.
+ */
+static XLogRecPtr
+ValidateInputLSN(XLogRecPtr lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate input WAL LSN. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(curr_lsn));
+
+	if (lsn >= curr_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot get information of WAL at future LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	return curr_lsn;
+}
+
+/*
+ * Validate given start LSN and end LSN, return the new end LSN in case user
+ * hasn't specified one.
+ */
+static XLogRecPtr
+ValidateStartAndEndLSNs(XLogRecPtr start_lsn, XLogRecPtr end_lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate WAL start LSN. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL start LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	if (start_lsn >= curr_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN cannot be a future WAL LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	/*
+	 * If end_lsn is specified, let's ensure that it's not a future LSN i.e.
+	 * something the database system doesn't know about.
+	 */
+	if (!XLogRecPtrIsInvalid(end_lsn) &&
+		(end_lsn >= curr_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL end LSN cannot be a future WAL LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	/*
+	 * When end_lsn is not specified let's read up to the last WAL position
+	 * known to be on the server.
+	 */
+	if (XLogRecPtrIsInvalid(end_lsn))
+		end_lsn = curr_lsn;
+
+	Assert(!XLogRecPtrIsInvalid(end_lsn));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	return end_lsn;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	/*
+	 * Display a message that we're skipping data if the given LSN wasn't a
+	 * pointer to the start of a record and also wasn't a pointer to the
+	 * beginning of a segment (e.g. we were used in file mode).
+	 */
+	if (*first_record != lsn && XLogSegmentOffset(lsn, wal_segment_size) != 0)
+		ereport(WARNING,
+				(errmsg_plural("first record is after %X/%X, at %X/%X, skipping over %u byte",
+							   "first record is after %X/%X, at %X/%X, skipping over %u bytes",
+							   (*first_record - lsn),
+							   LSN_FORMAT_ARGS(lsn),
+							   LSN_FORMAT_ARGS(*first_record),
+							   (uint32) (*first_record - lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get raw WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	*raw_record;
+	uint32	rec_len;
+	char	*raw_record_data;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = PointerGetDatum(raw_record);
+
+	Assert(i == PG_GET_RAW_WAL_RECORD_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_RAW_WAL_RECORD_COLS
+}
+
+/*
+ * Get first valid raw WAL record LSN following the given LSN.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+#define PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogRecPtr	prev_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS];
+	bool	nulls[PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	prev_record = XLogRecGetPrev(xlogreader);
+
+	XLogReaderFree(xlogreader);
+
+	/*
+	 * Previous valid WAL record must be at an LSN lower than next valid WAL
+	 * record LSN. Otherwise, it is an indication of something wrong, so error
+	 * out.
+	 */
+	if (prev_record >= first_record)
+		ereport(ERROR,
+				(errcode(ERRCODE_DATA_CORRUPTED),
+				 errmsg("record with incorrect prev-link %X/%X at %X/%X",
+				 LSN_FORMAT_ARGS(prev_record),
+				 LSN_FORMAT_ARGS(first_record))));
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(prev_record);
+
+	Assert(i == PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS
+}
+
+/*
+ * Calculate size of a record, split into !FPI and FPI parts.
+ */
+static void
+GetXLogRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
+{
+	int	block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data. It doesn't seem worth it to
+	 * add an accessor macro for this.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += record->blocks[block_id].bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	rec_len;
+	uint32	fpi_len;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int	block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(rec_len);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+
+	initStringInfo(&rec_desc);
+
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+
+	pfree(temp.data);
+
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		}
+		else
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+		}
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = PointerGetDatum(data);
+	values[i++] = UInt32GetDatum(main_data_len);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+					  PG_GET_WAL_RECORD_INFO_COLS);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+						  PG_GET_WAL_RECORDS_INFO_COLS);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future end WAL LSN i.e. WAL LSN the
+ * database system doesn't know about is specified.
+ *
+ * This function will figure out the end LSN if it's not specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	end_lsn = ValidateStartAndEndLSNs(start_lsn, end_lsn);
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+static void
+StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record)
+{
+	RmgrId	rmid;
+	uint8	recid;
+	uint32	rec_len;
+	uint32	fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics. */
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double	n_pct;
+	double	rec_len_pct;
+	double	fpi_len_pct;
+	double	tot_len_pct;
+	int	i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+					TupleDesc tupdesc, Datum *values, bool *nulls,
+					uint32 ncols)
+{
+	uint64	total_count = 0;
+	uint64	total_rec_len = 0;
+	uint64	total_fpi_len = 0;
+	uint64	total_len = 0;
+	int	ri;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls, ncols);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+					XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogRecStats stats;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_STATS_COLS];
+	bool	nulls[PG_GET_WAL_STATS_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		StoreXLogRecordStats(&stats, xlogreader);
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, tupstore, tupdesc, values, nulls,
+						PG_GET_WAL_STATS_COLS);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ *
+ * This function emits an error if a future end WAL LSN i.e. WAL LSN the
+ * database system doesn't know about is specified.
+ *
+ * This function will figure out the end LSN if it's not specified.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	end_lsn = ValidateStartAndEndLSNs(start_lsn, end_lsn);
+
+	GetWalStatsInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index 35029cf97d..4d217cabbe 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -956,13 +956,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1080,6 +1073,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 2340dc247b..60940d30c2 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 4b45ac64db..f34f228563 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 849954a8e5..38fd51bafa 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 477f0efe26..d62d6ce7f8 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

#42Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Bharath Rupireddy (#41)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Thanks Bharath for working on all my review comments. I took a quick
look at the new version of the patch (v7-pg_walinspect.patch) and this
version looks a lot better. I'll do some detailed review later (maybe
next week or so) and let you know my further comments, if any.

--
With Regards,
Ashutosh Sharma.

On Fri, Mar 4, 2022 at 3:54 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Show quoted text

On Thu, Mar 3, 2022 at 10:05 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Feb 25, 2022 at 6:03 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Added a new function that returns the first and last valid WAL record
LSN of a given WAL file.

Sounds like fuzzy thinking to me. WAL records can cross file
boundaries, and forgetting about that leads to all sorts of problems.
Just give people one function that decodes a range of LSNs and call it
good. Why do you need anything else? If people want to get the first
record that begins in a segment or the first record any portion of
which is in a particular segment or the last record that begins in a
segment or the last record that ends in a segment or any other such
thing, they can use a WHERE clause for that... and if you think they
can't, then that should be good cause to rethink the return value of
the one-and-only SRF that I think you need here.

Thanks Robert.

Thanks to others for your review comments.

Here's the v7 patch set. These patches are based on the motive that
"keep it simple and short yet effective and useful". With that in
mind, I have not implemented the wait mode for any of the functions
(as it doesn't look an effective use-case and requires adding a new
page_read callback, instead throw error if future LSN is specified),
also these functions will give WAL information on the current server's
timeline. Having said that, I'm open to adding new functions in future
once this initial version gets in, if there's a requirement and users
ask for the new functions.

Please review the v7 patch set and provide your thoughts.

Regards,
Bharath Rupireddy.

#43Jeff Davis
pgsql@j-davis.com
In reply to: Bharath Rupireddy (#35)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, 2022-03-02 at 22:37 +0530, Bharath Rupireddy wrote:

Attaching v6 patch set with above review comments addressed. Please
review it further.

* Don't issue WARNINGs or other messages for ordinary situations, like
when pg_get_wal_records_info() hits the end of WAL.

* It feels like the APIs that allow waiting for the end of WAL are
slightly off. Can't you just do pg_get_wal_records_info(start_lsn,
least(pg_current_wal_flush_lsn(), end_lsn)) if you want the non-waiting
behavior? Try to make the API more orthogonal, where a few basic
functions can be combined to give you everything you need, rather than
specifying extra parameters and issuing WARNINGs. I

* In the docs, include some example output. I don't see any output in
the tests, which makes sense because it's mostly non-deterministic, but
it would be helpful to see sample output of at least
pg_get_wal_records_info().

* Is pg_get_wal_stats() even necessary, or can you get the same
information with a query over pg_get_wal_records_info()? For instance,
if you want to group by transaction ID rather than rmgr, then
pg_get_wal_stats() is useless.

* Would be nice to have a pg_wal_file_is_valid() or similar, which
would test that it exists, and the header matches the filename (e.g. if
it was recycled but not used, that would count as invalid). I think
pg_get_first_valid_wal_record_lsn() would make some cases look invalid
even if the file is valid -- for example, if a wal record spans many
wal segments, the segments might look invalid because they contain no
complete records, but the file itself is still valid and contains valid
wal data.

* Is there a reason you didn't include the timeline ID in
pg_get_wal_records_info()?

* Can we mark this extension 'trusted'? I'm not 100% clear on the
standards for that marker, but it seems reasonable for a database owner
with the right privileges might want to install it.

* pg_get_raw_wal_record() seems too powerful for pg_monitor. Maybe that
function should require pg_read_server_files? Or at least
pg_read_all_data?

Regards,
Jeff Davis

#44Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Jeff Davis (#43)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Mar 10, 2022 at 1:52 PM Jeff Davis <pgsql@j-davis.com> wrote:

On Wed, 2022-03-02 at 22:37 +0530, Bharath Rupireddy wrote:

Attaching v6 patch set with above review comments addressed. Please
review it further.

Thanks Jeff for reviewing it. I've posted the latest v7 patch-set
upthread [1]/messages/by-id/CALj2ACWtToUQ5hCCBJP+mKeVUcN-g7cMb9XvhAcicPxUDsdcKg@mail.gmail.com which is having more simple-yet-useful-and-effective
functions.

* Don't issue WARNINGs or other messages for ordinary situations, like
when pg_get_wal_records_info() hits the end of WAL.

v7 patch-set [1]/messages/by-id/CALj2ACWtToUQ5hCCBJP+mKeVUcN-g7cMb9XvhAcicPxUDsdcKg@mail.gmail.com has no warnings, but the functions will error out if
future LSN is specified.

* It feels like the APIs that allow waiting for the end of WAL are
slightly off. Can't you just do pg_get_wal_records_info(start_lsn,
least(pg_current_wal_flush_lsn(), end_lsn)) if you want the non-waiting
behavior? Try to make the API more orthogonal, where a few basic
functions can be combined to give you everything you need, rather than
specifying extra parameters and issuing WARNINGs. I

v7 patch-set [1]/messages/by-id/CALj2ACWtToUQ5hCCBJP+mKeVUcN-g7cMb9XvhAcicPxUDsdcKg@mail.gmail.com onwards waiting mode has been removed for all of the
functions, again to keep things simple-yet-useful-and-effective.
However, we can always add new pg_walinspect functions that wait for
future WAL in the next versions once basic stuff gets committed and if
many users ask for it.

* In the docs, include some example output. I don't see any output in
the tests, which makes sense because it's mostly non-deterministic, but
it would be helpful to see sample output of at least
pg_get_wal_records_info().

+1. Added for pg_get_wal_records_info and pg_get_wal_stats.

* Is pg_get_wal_stats() even necessary, or can you get the same
information with a query over pg_get_wal_records_info()? For instance,
if you want to group by transaction ID rather than rmgr, then
pg_get_wal_stats() is useless.

Yes, you are right pg_get_wal_stats provides WAL stats per resource
manager which is similar to pg_waldump with --start, --end and --stats
option. It provides more information than pg_get_wal_records_info and
is a good way of getting stats than adding more columns to
pg_get_wal_records_info, calculating percentage in sql and having
group by clause. IMO, pg_get_wal_stats is more readable and useful.

* Would be nice to have a pg_wal_file_is_valid() or similar, which
would test that it exists, and the header matches the filename (e.g. if
it was recycled but not used, that would count as invalid). I think
pg_get_first_valid_wal_record_lsn() would make some cases look invalid
even if the file is valid -- for example, if a wal record spans many
wal segments, the segments might look invalid because they contain no
complete records, but the file itself is still valid and contains valid
wal data.

Actually I haven't tried testing a single WAL record spanning many WAL
files yet(I'm happy to try it if someone suggests such a use-case). In
that case too I assume pg_get_first_valid_wal_record_lsn() shouldn't
have a problem because it just gives the next valid LSN and it's
previous LSN using existing WAL reader API XLogFindNextRecord(). It
opens up the WAL file segments using (some dots to connect -
page_read/read_local_xlog_page, WALRead,
segment_open/wal_segment_open). Thoughts?

I don't think it's necessary to have a function pg_wal_file_is_valid()
that given a WAL file name as input checks whether a WAL file exists
or not, probably not in the core (xlogfuncs.c) too. These kinds of
functions can open up challenges in terms of user input validation and
may cause unnecessary problems, please see some related discussion
[2]: /messages/by-id/CA+TgmobYrTgMEF0SV+yDYyCCh44DAGjZVs7BYGrD8xD3vwNjHA@mail.gmail.com

* Is there a reason you didn't include the timeline ID in
pg_get_wal_records_info()?

I'm right now allowing the functions to read WAL from the current
server's timeline which I have mentioned in the docs. The server's
current timeline is available via pg_control_checkpoint()'s
timeline_id. So, having timeline_id as a column doesn't make sense.
Again this is to keep things simple-yet-useful-and-effective. However,
we can add new pg_walinspect functions to read WAL from historic as
well as current timelines in the next versions once basic stuff gets
committed and if many users ask for it.

+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>

* Can we mark this extension 'trusted'? I'm not 100% clear on the
standards for that marker, but it seems reasonable for a database owner
with the right privileges might want to install it.

'trusted' extensions concept is added by commit 50fc694 [3]commit 50fc694e43742ce3d04a5e9f708432cb022c5f0d Author: Tom Lane <tgl@sss.pgh.pa.us> Date: Wed Jan 29 18:42:43 2020 -0500. Since
pg_walinspect deals with WAL, we strictly want to control who creates
and can execute functions exposed by it, so I don't know if 'trusted'
is a good idea here. Also, pageinspect isn't a 'trusted' extension.

* pg_get_raw_wal_record() seems too powerful for pg_monitor. Maybe that
function should require pg_read_server_files? Or at least
pg_read_all_data?

pg_read_all_data may not be the right choice, but pg_read_server_files
is. However, does it sound good if some functions are allowed to be
executed by users with a pg_monitor role and others
pg_get_raw_wal_record by users with pg_read_server_files? Since the
extension itself can be created by superusers, isn't the
pg_get_raw_wal_record sort of safe with pg_mointor itself?

If hackers don't agree, I'm happy to grant execution on
pg_get_raw_wal_record() to the pg_read_server_files role.

Attaching the v8 patch-set resolving above comments and some tests for
checking function permissions. Please review it further.

[1]: /messages/by-id/CALj2ACWtToUQ5hCCBJP+mKeVUcN-g7cMb9XvhAcicPxUDsdcKg@mail.gmail.com
[2]: /messages/by-id/CA+TgmobYrTgMEF0SV+yDYyCCh44DAGjZVs7BYGrD8xD3vwNjHA@mail.gmail.com
[3]: commit 50fc694e43742ce3d04a5e9f708432cb022c5f0d Author: Tom Lane <tgl@sss.pgh.pa.us> Date: Wed Jan 29 18:42:43 2020 -0500
Author: Tom Lane <tgl@sss.pgh.pa.us>
Date: Wed Jan 29 18:42:43 2020 -0500

Invent "trusted" extensions, and remove the pg_pltemplate catalog.

Regards,
Bharath Rupireddy.

Attachments:

v8-0001-pg_walinspect.patchapplication/octet-stream; name=v8-0001-pg_walinspect.patchDownload
From 69408f5e8493c490b96c65fed46fc7f4d4954ea5 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 10 Mar 2022 16:26:04 +0000
Subject: [PATCH v8] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql |  96 ++
 contrib/pg_walinspect/pg_walinspect.c        | 949 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  14 +-
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/common/relpath.c                         |  18 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/common/relpath.h                 |   1 +
 13 files changed, 1114 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index e3e221308b..705c6fc36b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -40,6 +40,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..8720d3bb5e
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,96 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_first_valid_wal_record_lsn()
+--
+CREATE FUNCTION pg_get_first_valid_wal_record_lsn(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn
+)
+AS 'MODULE_PATHNAME', 'pg_get_first_valid_wal_record_lsn'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_first_valid_wal_record_lsn(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT length int4,
+    OUT total_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data bytea,
+    OUT data_len int4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..99f2f43c42
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,949 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct Stats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+}	Stats;
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	Stats		rmgr_stats[RM_NEXT_ID];
+	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+}	XLogRecStats;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_first_valid_wal_record_lsn);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+
+static XLogRecPtr ValidateInputLSN(XLogRecPtr lsn);
+static XLogRecPtr ValidateStartAndEndLSNs(XLogRecPtr start_lsn,
+										  XLogRecPtr end_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void StoreXLogRecordStats(XLogRecStats * stats,
+								 XLogReaderState *record);
+static void GetXLogSummaryStats(XLogRecStats * stats,
+								Tuplestorestate *tupstore, TupleDesc tupdesc,
+								Datum *values, bool *nulls, uint32 ncols);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+								XLogRecPtr end_lsn);
+static void GetWALRecordsInfoInternal(FunctionCallInfo fcinfo,
+									  XLogRecPtr start_lsn,
+									  XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Validate given LSN and return the LSN up to which the server has WAL.
+ */
+static XLogRecPtr
+ValidateInputLSN(XLogRecPtr lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate input WAL LSN. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(curr_lsn));
+
+	if (lsn >= curr_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot get information of WAL at future LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	return curr_lsn;
+}
+
+/*
+ * Validate given start LSN and end LSN, return the new end LSN in case user
+ * hasn't specified one.
+ */
+static XLogRecPtr
+ValidateStartAndEndLSNs(XLogRecPtr start_lsn, XLogRecPtr end_lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate WAL start LSN. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL start LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	if (start_lsn >= curr_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN cannot be a future WAL LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	/*
+	 * If end_lsn is specified, let's ensure that it's not a future LSN i.e.
+	 * something the database system doesn't know about.
+	 */
+	if (!XLogRecPtrIsInvalid(end_lsn) &&
+		(end_lsn >= curr_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL end LSN cannot be a future WAL LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	/*
+	 * When end_lsn is not specified let's read up to the last WAL position
+	 * known to be on the server.
+	 */
+	if (XLogRecPtrIsInvalid(end_lsn))
+		end_lsn = curr_lsn;
+
+	Assert(!XLogRecPtrIsInvalid(end_lsn));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	return end_lsn;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	/*
+	 * Display a message that we're skipping data if the given LSN wasn't a
+	 * pointer to the start of a record and also wasn't a pointer to the
+	 * beginning of a segment (e.g. we were used in file mode).
+	 */
+	if (*first_record != lsn && XLogSegmentOffset(lsn, wal_segment_size) != 0)
+		ereport(WARNING,
+				(errmsg_plural("first record is after %X/%X, at %X/%X, skipping over %u byte",
+							   "first record is after %X/%X, at %X/%X, skipping over %u bytes",
+							   (*first_record - lsn),
+							   LSN_FORMAT_ARGS(lsn),
+							   LSN_FORMAT_ARGS(*first_record),
+							   (uint32) (*first_record - lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get raw WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	*raw_record;
+	uint32	rec_len;
+	char	*raw_record_data;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = PointerGetDatum(raw_record);
+
+	Assert(i == PG_GET_RAW_WAL_RECORD_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_RAW_WAL_RECORD_COLS
+}
+
+/*
+ * Get first valid raw WAL record LSN following the given LSN.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_first_valid_wal_record_lsn(PG_FUNCTION_ARGS)
+{
+#define PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS 2
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogRecPtr	prev_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS];
+	bool	nulls[PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	prev_record = XLogRecGetPrev(xlogreader);
+
+	XLogReaderFree(xlogreader);
+
+	/*
+	 * Previous valid WAL record must be at an LSN lower than next valid WAL
+	 * record LSN. Otherwise, it is an indication of something wrong, so error
+	 * out.
+	 */
+	if (prev_record >= first_record)
+		ereport(ERROR,
+				(errcode(ERRCODE_DATA_CORRUPTED),
+				 errmsg("record with incorrect prev-link %X/%X at %X/%X",
+				 LSN_FORMAT_ARGS(prev_record),
+				 LSN_FORMAT_ARGS(first_record))));
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(prev_record);
+
+	Assert(i == PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_FIRST_VALID_WAL_RECORD_LSN_COLS
+}
+
+/*
+ * Calculate size of a record, split into !FPI and FPI parts.
+ */
+static void
+GetXLogRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
+{
+	int	block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data. It doesn't seem worth it to
+	 * add an accessor macro for this.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += record->blocks[block_id].bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	rec_len;
+	uint32	fpi_len;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int	block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(rec_len);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+
+	initStringInfo(&rec_desc);
+
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+
+	pfree(temp.data);
+
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		}
+		else
+		{
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+		}
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = PointerGetDatum(data);
+	values[i++] = UInt32GetDatum(main_data_len);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 10
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+					  PG_GET_WAL_RECORD_INFO_COLS);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 10
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+						  PG_GET_WAL_RECORDS_INFO_COLS);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future end WAL LSN i.e. WAL LSN the
+ * database system doesn't know about is specified.
+ *
+ * This function will figure out the end LSN if it's not specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	end_lsn = ValidateStartAndEndLSNs(start_lsn, end_lsn);
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+static void
+StoreXLogRecordStats(XLogRecStats * stats, XLogReaderState *record)
+{
+	RmgrId	rmid;
+	uint8	recid;
+	uint32	rec_len;
+	uint32	fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	GetXLogRecordLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics. */
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double	n_pct;
+	double	rec_len_pct;
+	double	fpi_len_pct;
+	double	tot_len_pct;
+	int	i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogRecStats * stats, Tuplestorestate *tupstore,
+					TupleDesc tupdesc, Datum *values, bool *nulls,
+					uint32 ncols)
+{
+	uint64	total_count = 0;
+	uint64	total_rec_len = 0;
+	uint64	total_fpi_len = 0;
+	uint64	total_len = 0;
+	int	ri;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls, ncols);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStatsInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+					XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogRecStats stats;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_STATS_COLS];
+	bool	nulls[PG_GET_WAL_STATS_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		StoreXLogRecordStats(&stats, xlogreader);
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, tupstore, tupdesc, values, nulls,
+						PG_GET_WAL_STATS_COLS);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ *
+ * This function emits an error if a future end WAL LSN i.e. WAL LSN the
+ * database system doesn't know about is specified.
+ *
+ * This function will figure out the end LSN if it's not specified.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	end_lsn = ValidateStartAndEndLSNs(start_lsn, end_lsn);
+
+	GetWalStatsInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index 35029cf97d..4d217cabbe 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -956,13 +956,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1080,6 +1073,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 2340dc247b..60940d30c2 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 4b45ac64db..f34f228563 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 849954a8e5..38fd51bafa 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 477f0efe26..d62d6ce7f8 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v8-0001-pg_walinspect-tests.patchapplication/octet-stream; name=v8-0001-pg_walinspect-tests.patchDownload
From dfd178dc8417bc8ce3fe8aa7ddfb28ddf5a9c152 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 10 Mar 2022 16:41:26 +0000
Subject: [PATCH v8] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 126 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   |  99 ++++++++++++++
 2 files changed, 225 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..b8b9f095c6
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,126 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT lsn AS valid_wal_lsn1 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1') \gset
+SELECT lsn AS valid_wal_lsn2 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn2') \gset
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+GRANT pg_monitor TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..5b7196d9ca
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,99 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_first_valid_wal_record_lsn('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1');
+
+SELECT lsn AS valid_wal_lsn1 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn1') \gset
+
+SELECT lsn AS valid_wal_lsn2 FROM pg_get_first_valid_wal_record_lsn(:'wal_lsn2') \gset
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'valid_wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'valid_wal_lsn1', :'valid_wal_lsn2');
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'valid_wal_lsn1', :'valid_wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+GRANT pg_monitor TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v8-0001-pg_walinspect-docs.patchapplication/octet-stream; name=v8-0001-pg_walinspect-docs.patchDownload
From 157777bbf29f21f26d2f969aad2acae05aeb500d Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 10 Mar 2022 16:43:15 +0000
Subject: [PATCH v8] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 173 +++++++++++++++++++++++++++++++++
 3 files changed, 175 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index be9711c6f2..19614a42e1 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -130,6 +130,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index 328cd1f378..a2e8fd4a08 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -146,6 +146,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..9c61de1838
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,173 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_monitor</literal> role. Access may be granted to others using
+  <command>GRANT</command>.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>pg_get_raw_wal_record(in_lsn pg_lsn, lsn OUT pg_lsn, record OUT bytea)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. Issues a warning if the given
+      LSN wasn't a pointer to the start of a record and also wasn't a pointer
+      to the beginning of a WAL segment file. This function emits an error if
+      a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_first_valid_wal_record_lsn(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets first and previous valid WAL record LSNs of the given LSN. Issues
+      a warning if the given LSN wasn't a pointer to the start of a record and
+      also wasn't a pointer to the beginning of a WAL segment file. This
+      function emits an error if a future (the LSN database system doesn't know
+      about) <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>pg_get_wal_record_info(in_lsn pg_lsn, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of the given LSN. Issues a warning if the
+      given LSN wasn't a pointer to the start of a record and also wasn't a
+      pointer to the beginning of a WAL segment file. This function emits an
+      error if a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_records_info(start_lsn pg_lsn, end_lsn pg_lsn DEFAULT NULL, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length OUT int4, total_length OUT int4, description OUT text, block_ref OUT text, data OUT bytea, data_len OUT int4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. Issues a warning if the given
+      <replaceable>start_lsn</replaceable> wasn't a pointer to the start of a
+      record and also wasn't a pointer to the beginning of a WAL segment file.
+      This function figures out the <replaceable>end_lsn</replaceable> if it's
+      not specified, that means, it returns information up to the end of WAL.
+      Default value of <replaceable>end_lsn</replaceable> is <literal>NULL</literal>.
+      This function emits an error if a future (the LSN database system doesn't
+      know about) <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example:
+<screen>
+postgres=# select prev_lsn, xid, resource_manager, length, total_length, block_ref from pg_get_wal_records_info('0/158A7F0', '0/1591400');
+ prev_lsn  | xid | resource_manager | length | total_length |                                block_ref                                 
+-----------+-----+------------------+--------+--------------+--------------------------------------------------------------------------
+ 0/158A7B8 | 735 | Heap             |     54 |         7838 | blkref #0: rel 1663/5/2619 blk 18 (FPW); hole: offset: 88, length: 408
+ 0/158A7F0 | 735 | Btree            |     53 |         8133 | blkref #0: rel 1663/5/2696 blk 1 (FPW); hole: offset: 1632, length: 112
+ 0/158C6A8 | 735 | Heap             |     53 |          873 | blkref #0: rel 1663/5/1259 blk 0 (FPW); hole: offset: 212, length: 7372
+ 0/158E688 | 735 | Transaction      |    114 |          114 | 
+ 0/158E9F8 |   0 | Standby          |     50 |           50 | 
+ 0/158EA70 |   0 | Heap2            |     59 |         6755 | blkref #0: rel 1663/5/1255 blk 92 (FPW); hole: offset: 120, length: 1496
+ 0/158EAA8 |   0 | Standby          |     50 |           50 | 
+ 0/1590528 | 736 | Heap             |     54 |         3570 | blkref #0: rel 1663/5/16411 blk 4 (FPW); hole: offset: 412, length: 4676
+ 0/1590560 | 736 | Heap             |     59 |           59 | blkref #0: rel 1663/5/16411 blk 4
+ 0/1591358 | 736 | Heap             |     59 |           59 | blkref #0: rel 1663/5/16411 blk 4
+ 0/1591398 | 736 | Transaction      |     34 |           34 | 
+(11 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>pg_get_wal_stats(start_lsn pg_lsn, end_lsn pg_lsn, wait_for_wal boolean DEFAULT false, resource_manager OUT text, count OUT int8, count_percentage OUT float4, record_size OUT int8, record_size_percentage OUT float4, fpi_size OUT int8, fpi_size_percentage OUT float4, combined_size OUT int8, combined_size_percentage OUT float4)</function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each <replaceable>resource_manager</replaceable>
+      type. Issues a warning if the given <replaceable>start_lsn</replaceable>
+      wasn't a pointer to the start of a record and also wasn't a pointer to
+      the beginning of a WAL segment file. This function figures out the
+      <replaceable>end_lsn</replaceable> if it's not specified, that means, it
+      returns information up to the end of WAL. Default value of
+      <replaceable>end_lsn</replaceable> is <literal>NULL</literal>. This
+      function emits an error if a future (the LSN database system doesn't know
+      about) <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/158A7F0', '0/1591400');
+ resource_manager  | count | count_percentage | record_size | record_size_percentage | fpi_size | fpi_size_percentage | combined_size | combined_size_percentage 
+-------------------+-------+------------------+-------------+------------------------+----------+---------------------+---------------+--------------------------
+ XLOG              |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ Transaction       |     2 |        18.181818 |         148 |               23.16119 |        0 |                   0 |           148 |               0.53749776
+ Storage           |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ CLOG              |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ Database          |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ Tablespace        |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ MultiXact         |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ RelMap            |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ Standby           |     2 |        18.181818 |         100 |              15.649452 |        0 |                   0 |           100 |               0.36317414
+ Heap2             |     1 |         9.090909 |          59 |               9.233177 |     6696 |           24.895895 |          6755 |                24.532413
+ Heap              |     5 |        45.454544 |         279 |              43.661972 |    12120 |            45.06246 |         12399 |                 45.02996
+ Btree             |     1 |         9.090909 |          53 |              8.2942095 |     8080 |           30.041641 |          8133 |                29.536953
+ Hash              |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ Gin               |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ Gist              |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ Sequence          |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ SPGist            |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ BRIN              |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ CommitTs          |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ ReplicationOrigin |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ Generic           |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+ LogicalMessage    |     0 |                0 |           0 |                      0 |        0 |                   0 |             0 |                        0
+(22 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#45Robert Haas
robertmhaas@gmail.com
In reply to: Jeff Davis (#43)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Mar 10, 2022 at 3:22 AM Jeff Davis <pgsql@j-davis.com> wrote:

* Can we mark this extension 'trusted'? I'm not 100% clear on the
standards for that marker, but it seems reasonable for a database owner
with the right privileges might want to install it.

I'm not clear on the standard either, exactly, but might not that
allow the database owner to get a peek at what's happening in other
databases?

--
Robert Haas
EDB: http://www.enterprisedb.com

#46Stephen Frost
sfrost@snowman.net
In reply to: Robert Haas (#45)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Greetings,

* Robert Haas (robertmhaas@gmail.com) wrote:

On Thu, Mar 10, 2022 at 3:22 AM Jeff Davis <pgsql@j-davis.com> wrote:

* Can we mark this extension 'trusted'? I'm not 100% clear on the
standards for that marker, but it seems reasonable for a database owner
with the right privileges might want to install it.

I'm not clear on the standard either, exactly, but might not that
allow the database owner to get a peek at what's happening in other
databases?

The standard is basically that all of the functions it brings are
written to enforce the PG privilege system and you aren't able to use
the extension to bypass those privileges. In some cases that means that
the C-language functions installed have if(!superuser) ereport() calls
throughout them- that's a fine answer, but it's perhaps not very helpful
to mark those as trusted. In other cases, the C-language functions
installed don't directly provide access to data, such as the PostGIS
functions.

I've not looked back on this thread, but I'd expect pg_walinspect to
need those superuser checks and with those it *could* be marked as
trusted, but that again brings into question how useful it is to mark it
thusly.

In an ideal world, we might have a pg_readwal predefined role which
allows a role which was GRANT'd that role to be able to read WAL
traffic, and then the pg_walinspect extension could check that the
calling role has that predefined role, and other functions and
extensions could also check that rather than any existing superuser
checks. A cloud provider or such could then include in their setup of a
new instance something like:

GRANT pg_readwal TO admin_user WITH ADMIN OPTION;

Presuming that there isn't anything that ends up in the WAL that's an
issue for the admin_user to have access to.

I certainly don't think we should allow either database owners or
regular users on a system the ability to access the WAL traffic of the
entire system. More forcefully- we should *not* be throwing more access
rights towards $owners in general and should be thinking about how we
can allow admins, providers, whomever, the ability to control what
rights users are given. If they're all lumped under 'owner' then
there's no way for people to provide granular access to just those
things they wish and intend to.

Thanks,

Stephen

#47Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Bharath Rupireddy (#44)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

At Thu, 10 Mar 2022 22:15:42 +0530, Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com> wrote in

On Thu, Mar 10, 2022 at 1:52 PM Jeff Davis <pgsql@j-davis.com> wrote:

On Wed, 2022-03-02 at 22:37 +0530, Bharath Rupireddy wrote:

Attaching v6 patch set with above review comments addressed. Please
review it further.

Thanks Jeff for reviewing it. I've posted the latest v7 patch-set
upthread [1] which is having more simple-yet-useful-and-effective
functions.

* Don't issue WARNINGs or other messages for ordinary situations, like
when pg_get_wal_records_info() hits the end of WAL.

v7 patch-set [1] has no warnings, but the functions will error out if
future LSN is specified.

* It feels like the APIs that allow waiting for the end of WAL are
slightly off. Can't you just do pg_get_wal_records_info(start_lsn,
least(pg_current_wal_flush_lsn(), end_lsn)) if you want the non-waiting
behavior? Try to make the API more orthogonal, where a few basic
functions can be combined to give you everything you need, rather than
specifying extra parameters and issuing WARNINGs. I

v7 patch-set [1] onwards waiting mode has been removed for all of the
functions, again to keep things simple-yet-useful-and-effective.
However, we can always add new pg_walinspect functions that wait for
future WAL in the next versions once basic stuff gets committed and if
many users ask for it.

* In the docs, include some example output. I don't see any output in
the tests, which makes sense because it's mostly non-deterministic, but
it would be helpful to see sample output of at least
pg_get_wal_records_info().

+1. Added for pg_get_wal_records_info and pg_get_wal_stats.

* Is pg_get_wal_stats() even necessary, or can you get the same
information with a query over pg_get_wal_records_info()? For instance,
if you want to group by transaction ID rather than rmgr, then
pg_get_wal_stats() is useless.

Yes, you are right pg_get_wal_stats provides WAL stats per resource
manager which is similar to pg_waldump with --start, --end and --stats
option. It provides more information than pg_get_wal_records_info and
is a good way of getting stats than adding more columns to
pg_get_wal_records_info, calculating percentage in sql and having
group by clause. IMO, pg_get_wal_stats is more readable and useful.

* Would be nice to have a pg_wal_file_is_valid() or similar, which
would test that it exists, and the header matches the filename (e.g. if
it was recycled but not used, that would count as invalid). I think
pg_get_first_valid_wal_record_lsn() would make some cases look invalid
even if the file is valid -- for example, if a wal record spans many
wal segments, the segments might look invalid because they contain no
complete records, but the file itself is still valid and contains valid
wal data.

Actually I haven't tried testing a single WAL record spanning many WAL
files yet(I'm happy to try it if someone suggests such a use-case). In
that case too I assume pg_get_first_valid_wal_record_lsn() shouldn't
have a problem because it just gives the next valid LSN and it's
previous LSN using existing WAL reader API XLogFindNextRecord(). It
opens up the WAL file segments using (some dots to connect -
page_read/read_local_xlog_page, WALRead,
segment_open/wal_segment_open). Thoughts?

I don't think it's necessary to have a function pg_wal_file_is_valid()
that given a WAL file name as input checks whether a WAL file exists
or not, probably not in the core (xlogfuncs.c) too. These kinds of
functions can open up challenges in terms of user input validation and
may cause unnecessary problems, please see some related discussion
[2].

* Is there a reason you didn't include the timeline ID in
pg_get_wal_records_info()?

I'm right now allowing the functions to read WAL from the current
server's timeline which I have mentioned in the docs. The server's
current timeline is available via pg_control_checkpoint()'s
timeline_id. So, having timeline_id as a column doesn't make sense.
Again this is to keep things simple-yet-useful-and-effective. However,
we can add new pg_walinspect functions to read WAL from historic as
well as current timelines in the next versions once basic stuff gets
committed and if many users ask for it.

+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>

* Can we mark this extension 'trusted'? I'm not 100% clear on the
standards for that marker, but it seems reasonable for a database owner
with the right privileges might want to install it.

'trusted' extensions concept is added by commit 50fc694 [3]. Since
pg_walinspect deals with WAL, we strictly want to control who creates
and can execute functions exposed by it, so I don't know if 'trusted'
is a good idea here. Also, pageinspect isn't a 'trusted' extension.

* pg_get_raw_wal_record() seems too powerful for pg_monitor. Maybe that
function should require pg_read_server_files? Or at least
pg_read_all_data?

pg_read_all_data may not be the right choice, but pg_read_server_files
is. However, does it sound good if some functions are allowed to be
executed by users with a pg_monitor role and others
pg_get_raw_wal_record by users with pg_read_server_files? Since the
extension itself can be created by superusers, isn't the
pg_get_raw_wal_record sort of safe with pg_mointor itself?

If hackers don't agree, I'm happy to grant execution on
pg_get_raw_wal_record() to the pg_read_server_files role.

Attaching the v8 patch-set resolving above comments and some tests for
checking function permissions. Please review it further.

[1] /messages/by-id/CALj2ACWtToUQ5hCCBJP+mKeVUcN-g7cMb9XvhAcicPxUDsdcKg@mail.gmail.com
[2] /messages/by-id/CA+TgmobYrTgMEF0SV+yDYyCCh44DAGjZVs7BYGrD8xD3vwNjHA@mail.gmail.com
[3] commit 50fc694e43742ce3d04a5e9f708432cb022c5f0d
Author: Tom Lane <tgl@sss.pgh.pa.us>
Date: Wed Jan 29 18:42:43 2020 -0500

Invent "trusted" extensions, and remove the pg_pltemplate catalog.

I played with this a bit, and would like to share some thoughts on it.

It seems to me too rigorous that pg_get_wal_records_info/stats()
reject future LSNs as end-LSN and I think WARNING or INFO and stop at
the real end-of-WAL is more kind to users. I think the same with the
restriction that start and end LSN are required to be different.

The definition of end-lsn is fuzzy here. If I fed a future LSN to the
functions, they tell me the beginning of the current insertion point
in error message. On the other hand they don't accept the same
value as end-LSN. I think it is right that they tell the current
insertion point and they should take the end-LSN as the LSN to stop
reading.

I think pg_get_wal_stats() is worth to have but I think it should be
implemented in SQL. Currently pg_get_wal_records_info() doesn't tell
about FPI since pg_waldump doesn't but it is internally collected (of
course!) and easily revealed. If we do that, the
pg_get_wal_records_stats() would be reduced to the following SQL
statement

SELECT resource_manager resmgr,
count(*) AS N,
(count(*) * 100 / sum(count(*)) OVER tot)::numeric(5,2) AS "%N",
sum(total_length) AS "combined size",
(sum(total_length) * 100 / sum(sum(total_length)) OVER tot)::numeric(5,2) AS "%combined size",
sum(fpi_len) AS fpilen,
(sum(fpi_len) * 100 / sum(sum(fpi_len)) OVER tot)::numeric(5,2) AS "%fpilen"
FROM pg_get_wal_records_info('0/1000000', '0/175DD7f')
GROUP by resource_manager
WINDOW tot AS ()
ORDER BY "combined size" desc;

The only difference with pg_waldump is the statement above doesn't
show lines for the resource managers that don't contained in the
result of pg_get_wal_records_info(). But I don't think that matters.

Sometimes the field description has very long (28kb long) content. It
makes the result output almost unreadable and I had a bit hard time
struggling with the output full of '-'s. I would like have a default
limit on the length of such fields that can be long but I'm not sure
we want that.

The difference between pg_get_wal_record_info and _records_ other than
the number of argument is the former accepts incorrect LSNs.

The following works,
pg_get_wal_record_info('0/1000000');
pg_get_wal_records_info('0/1000000');

but this doesn't
pg_get_wal_records_info('0/1000000', '0/1000000');

ERROR: WAL start LSN must be less than end LSN

But the following works
pg_get_wal_records_info('0/1000000', '0/1000029');

0/1000028 | 0/0 | 0

So I think we can consolidate the two functions as:

- pg_get_wal_records_info('0/1000000');

(current behavior) find the first record and show all records
thereafter.

- pg_get_wal_records_info('0/1000000', '0/1000000');

finds the first record since the start lsn and show it.

- pg_get_wal_records_info('0/1000000', '0/1000030');

finds the first record since the start lsn then show records up to
the end-lsn.

And about pg_get_raw_wal_record(). I don't see any use-case of the
function alone on SQL interface. Even if we need to inspect broken
WAL files, it needs profound knowledge of WAL format and tools that
doesn't work on SQL interface.

However like pageinspect, if we separate the WAL-record fetching and
parsing it could be thought as useful.

pg_get_wal_records_info woule be like:

SELECT * FROM pg_walinspect_parse(raw)
FROM (SELECT * FROM pg_walinspect_get_raw(start_lsn, end_lsn));

And pg_get_wal_stats woule be like:

SELECT * FROM pg_walinpect_stat(pg_walinspect_parse(raw))
FROM (SELECT * FROM pg_walinspect_get_raw(start_lsn, end_lsn)));

Regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#48Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#47)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Sorry, some minor non-syntactical corrections.

At Fri, 11 Mar 2022 11:38:22 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

I played with this a bit, and would like to share some thoughts on it.

It seems to me too rigorous that pg_get_wal_records_info/stats()
reject future LSNs as end-LSN and I think WARNING or INFO and stop at
the real end-of-WAL is more kind to users. I think the same with the
restriction that start and end LSN are required to be different.

The definition of end-lsn is fuzzy here. If I fed a future LSN to the
functions, they tell me the beginning of the current insertion point
in error message. On the other hand they don't accept the same
value as end-LSN. I think it is right that they tell the current
insertion point and they should take the end-LSN as the LSN to stop
reading.

I think pg_get_wal_stats() is worth to have but I think it should be
implemented in SQL. Currently pg_get_wal_records_info() doesn't tell
about FPI since pg_waldump doesn't but it is internally collected (of
course!) and easily revealed. If we do that, the
pg_get_wal_records_stats() would be reduced to the following SQL
statement

SELECT resource_manager resmgr,
count(*) AS N,
(count(*) * 100 / sum(count(*)) OVER tot)::numeric(5,2) AS "%N",
sum(total_length) AS "combined size",
(sum(total_length) * 100 / sum(sum(total_length)) OVER tot)::numeric(5,2) AS "%combined size",
sum(fpi_len) AS fpilen,
(sum(fpi_len) * 100 / sum(sum(fpi_len)) OVER tot)::numeric(5,2) AS "%fpilen"
FROM pg_get_wal_records_info('0/1000000', '0/175DD7f')
GROUP by resource_manager
WINDOW tot AS ()
ORDER BY "combined size" desc;

The only difference with pg_waldump is the statement above doesn't
show lines for the resource managers that don't contained in the
result of pg_get_wal_records_info(). But I don't think that matters.

Sometimes the field description has very long (28kb long) content. It
makes the result output almost unreadable and I had a bit hard time
struggling with the output full of '-'s. I would like have a default
limit on the length of such fields that can be long but I'm not sure
we want that.

- The difference between pg_get_wal_record_info and _records_ other than
- the number of argument is the former accepts incorrect LSNs.

The discussion is somewhat confused after some twists and turns.. It
should be something like the following.

pg_get_wal_record_info and pg_get_wal_records_info are almost same
since the latter can show a single record. However it is a bit
annoying to do that. Since, other than it doens't accept same LSNs for
start and end, it doesn't show a record when there' no record in the
specfied LSN range. But I don't think there's no usefulness of the
behavior.

The following works,
pg_get_wal_record_info('0/1000000');
pg_get_wal_records_info('0/1000000');

but this doesn't
pg_get_wal_records_info('0/1000000', '0/1000000');

ERROR: WAL start LSN must be less than end LSN

And the following shows no records.
pg_get_wal_records_info('0/1000000', '0/1000001');
pg_get_wal_records_info('0/1000000', '0/1000028');

But the following works
pg_get_wal_records_info('0/1000000', '0/1000029');

0/1000028 | 0/0 | 0

So I think we can consolidate the two functions as:

- pg_get_wal_records_info('0/1000000');

(current behavior) find the first record and show all records
thereafter.

- pg_get_wal_records_info('0/1000000', '0/1000000');

finds the first record since the start lsn and show it.

- pg_get_wal_records_info('0/1000000', '0/1000030');

finds the first record since the start lsn then show records up to
the end-lsn.

And about pg_get_raw_wal_record(). I don't see any use-case of the
function alone on SQL interface. Even if we need to inspect broken
WAL files, it needs profound knowledge of WAL format and tools that
doesn't work on SQL interface.

However like pageinspect, if we separate the WAL-record fetching and
parsing it could be thought as useful.

pg_get_wal_records_info woule be like:

SELECT * FROM pg_walinspect_parse(raw)
FROM (SELECT * FROM pg_walinspect_get_raw(start_lsn, end_lsn));

And pg_get_wal_stats woule be like:

SELECT * FROM pg_walinpect_stat(pg_walinspect_parse(raw))
FROM (SELECT * FROM pg_walinspect_get_raw(start_lsn, end_lsn)));

Regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#49Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Kyotaro Horiguchi (#48)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Mar 11, 2022 at 8:22 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Sorry, some minor non-syntactical corrections.

At Fri, 11 Mar 2022 11:38:22 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

I played with this a bit, and would like to share some thoughts on it.

It seems to me too rigorous that pg_get_wal_records_info/stats()
reject future LSNs as end-LSN and I think WARNING or INFO and stop at
the real end-of-WAL is more kind to users. I think the same with the
restriction that start and end LSN are required to be different.

The definition of end-lsn is fuzzy here. If I fed a future LSN to the
functions, they tell me the beginning of the current insertion point
in error message. On the other hand they don't accept the same
value as end-LSN. I think it is right that they tell the current
insertion point and they should take the end-LSN as the LSN to stop
reading.

I think pg_get_wal_stats() is worth to have but I think it should be
implemented in SQL. Currently pg_get_wal_records_info() doesn't tell
about FPI since pg_waldump doesn't but it is internally collected (of
course!) and easily revealed. If we do that, the
pg_get_wal_records_stats() would be reduced to the following SQL
statement

SELECT resource_manager resmgr,
count(*) AS N,
(count(*) * 100 / sum(count(*)) OVER tot)::numeric(5,2) AS "%N",
sum(total_length) AS "combined size",
(sum(total_length) * 100 / sum(sum(total_length)) OVER tot)::numeric(5,2) AS "%combined size",
sum(fpi_len) AS fpilen,
(sum(fpi_len) * 100 / sum(sum(fpi_len)) OVER tot)::numeric(5,2) AS "%fpilen"
FROM pg_get_wal_records_info('0/1000000', '0/175DD7f')
GROUP by resource_manager
WINDOW tot AS ()
ORDER BY "combined size" desc;

The only difference with pg_waldump is the statement above doesn't
show lines for the resource managers that don't contained in the
result of pg_get_wal_records_info(). But I don't think that matters.

Sometimes the field description has very long (28kb long) content. It
makes the result output almost unreadable and I had a bit hard time
struggling with the output full of '-'s. I would like have a default
limit on the length of such fields that can be long but I'm not sure
we want that.

- The difference between pg_get_wal_record_info and _records_ other than
- the number of argument is the former accepts incorrect LSNs.

The discussion is somewhat confused after some twists and turns.. It
should be something like the following.

pg_get_wal_record_info and pg_get_wal_records_info are almost same
since the latter can show a single record. However it is a bit
annoying to do that. Since, other than it doens't accept same LSNs for
start and end, it doesn't show a record when there' no record in the
specfied LSN range. But I don't think there's no usefulness of the
behavior.

The following works,
pg_get_wal_record_info('0/1000000');

This does work but it doesn't show any WARNING message for the start
pointer adjustment. I think it should.

pg_get_wal_records_info('0/1000000');

I think this is fine. It should be working because the user hasn't
specified the end pointer so we assume the default end pointer is
end-of-WAL.

but this doesn't
pg_get_wal_records_info('0/1000000', '0/1000000');

ERROR: WAL start LSN must be less than end LSN

I think this behaviour is fine. We cannot have the same start and end
lsn pointers.

And the following shows no records.
pg_get_wal_records_info('0/1000000', '0/1000001');
pg_get_wal_records_info('0/1000000', '0/1000028');

I think we should be erroring out here saying - couldn't find any
valid WAL record between given start and end lsn because there exists
no valid wal records between the specified start and end lsn pointers.

--
With Regards,
Ashutosh Sharma.

#50Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Kyotaro Horiguchi (#48)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Mar 11, 2022 at 8:22 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

- The difference between pg_get_wal_record_info and _records_ other than
- the number of argument is the former accepts incorrect LSNs.

The discussion is somewhat confused after some twists and turns.. It
should be something like the following.

pg_get_wal_record_info and pg_get_wal_records_info are almost same
since the latter can show a single record. However it is a bit
annoying to do that. Since, other than it doens't accept same LSNs for
start and end, it doesn't show a record when there' no record in the
specfied LSN range. But I don't think there's no usefulness of the
behavior.

So, do you want the pg_get_wal_record_info function to be removed as
we can use pg_get_wal_records_info() to achieve what it does?

--
With Regards,
Ashutosh Sharma.

#51Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Kyotaro Horiguchi (#47)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Mar 11, 2022 at 8:08 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

Attaching the v8 patch-set resolving above comments and some tests for
checking function permissions. Please review it further.

I played with this a bit, and would like to share some thoughts on it.

Thanks a lot Kyotaro-san for reviewing.

It seems to me too rigorous that pg_get_wal_records_info/stats()
reject future LSNs as end-LSN and I think WARNING or INFO and stop at
the real end-of-WAL is more kind to users. I think the same with the
restriction that start and end LSN are required to be different.

Throwing error on future LSNs is the same behaviour for all of the
pg_walinspect function input LSNs. IMO it is a cleaner thing to do
rather than confuse the users with different behaviours for each
function. The principle is this - pg_walinspect functions can't show
future WAL info. Having said that, I agree to make it a WARNING
instead of ERROR, for the simple reason that ERROR aborts the txn and
the applications can retry without aborting the txn. For instance,
pg_terminate_backend emits a WARNING if the PID isn't a postgres
process id.

PS: WARNING may not be a better idea than ERROR if we turn
pg_get_wal_stats a SQL function, see my response below.

The definition of end-lsn is fuzzy here. If I fed a future LSN to the
functions, they tell me the beginning of the current insertion point
in error message. On the other hand they don't accept the same
value as end-LSN. I think it is right that they tell the current
insertion point and they should take the end-LSN as the LSN to stop
reading.

The future LSN is determined by this:

if (!RecoveryInProgress())
available_lsn = GetFlushRecPtr(NULL);
else
available_lsn = GetXLogReplayRecPtr(NULL);

GetFlushRecPtr returns last byte + 1 flushed meaning this is the end
LSN currently known in the server, but it is not the start LSN of the
last WAL record in the server. Same goes with GetXLogReplayRecPtr
which gives lastReplayedEndRecPtr end+1 position. I picked
GetFlushRecPtr and GetXLogReplayRecPtr to determine the future WAL LSN
because this is how read_local_xlog_page determines to read WAL upto
and goes for wait mode, but I wanted to avoid the wait mode completely
for all the pg_walinspect functions (to keep things simple for now),
hence doing the similar checks within the input validation code and
emitting warning.

And you are right when we emit something like below, users tend to use
0/15B6D68 (from the DETAIL message) as the end LSN. I don't want to
ignore this DETAIL message altogether as it gives an idea where the
server is. How about rephrasing the DETAIL message a bit, something
like "Database system flushed the WAL up to WAL LSN %X/% X.'' or some
other better phrasing?

WARNING: WAL start LSN cannot be a future WAL LSN
DETAIL: Last known WAL LSN on the database system is 0/15B6D68.

If the users aren't sure about what's the end record LSN, they can
just use pg_get_wal_records_info and pg_get_wal_stats without end LSN:
select * from pg_get_wal_records_info('0/15B6D68');
select * from pg_get_wal_stats('0/15B6D68');

I think pg_get_wal_stats() is worth to have but I think it should be
implemented in SQL. Currently pg_get_wal_records_info() doesn't tell
about FPI since pg_waldump doesn't but it is internally collected (of
course!) and easily revealed. If we do that, the
pg_get_wal_records_stats() would be reduced to the following SQL
statement

SELECT resource_manager resmgr,
count(*) AS N,
(count(*) * 100 / sum(count(*)) OVER tot)::numeric(5,2) AS "%N",
sum(total_length) AS "combined size",
(sum(total_length) * 100 / sum(sum(total_length)) OVER tot)::numeric(5,2) AS "%combined size",
sum(fpi_len) AS fpilen,
(sum(fpi_len) * 100 / sum(sum(fpi_len)) OVER tot)::numeric(5,2) AS "%fpilen"
FROM pg_get_wal_records_info('0/1000000', '0/175DD7f')
GROUP by resource_manager
WINDOW tot AS ()
ORDER BY "combined size" desc;

The only difference with pg_waldump is the statement above doesn't
show lines for the resource managers that don't contained in the
result of pg_get_wal_records_info(). But I don't think that matters.

Yeah, this is better. One problem with the above is when
pg_get_wal_records_info emits a warning for future LSN. But this
shouldn't stop us doing it via SQL. Instead I would let all the
pg_walinspect functions emit errors as opposed to WARNING. Thoughts?

postgres=# SELECT resource_manager, count(*) AS count,
(count(*) * 100 / sum(count(*)) OVER tot)::numeric(5,2) AS count_percentage,
sum(total_length) AS combined_size,
(sum(total_length) * 100 / sum(sum(total_length)) OVER
tot)::numeric(5,2) AS combined_size_percentage
FROM pg_get_wal_records_info('0/10A3E50', '0/25B6F00')
GROUP BY resource_manager
WINDOW tot AS ()
ORDER BY combined_size desc;
WARNING: WAL end LSN cannot be a future WAL LSN
DETAIL: Last known WAL LSN on the database system is 0/15CAA70.
resource_manager | count | count_percentage | combined_size |
combined_size_percentage
------------------+-------+------------------+---------------+--------------------------
| 1 | 100.00 | |
(1 row)

Sometimes the field description has very long (28kb long) content. It
makes the result output almost unreadable and I had a bit hard time
struggling with the output full of '-'s. I would like have a default
limit on the length of such fields that can be long but I'm not sure
we want that.

Yeah, it's a text column, let's leave it as-is, if required users can
always ignore the description columns.

And about pg_get_raw_wal_record(). I don't see any use-case of the
function alone on SQL interface. Even if we need to inspect broken
WAL files, it needs profound knowledge of WAL format and tools that
doesn't work on SQL interface.
However like pageinspect, if we separate the WAL-record fetching and
parsing it could be thought as useful.
SELECT * FROM pg_walinspect_parse(raw)
FROM (SELECT * FROM pg_walinspect_get_raw(start_lsn, end_lsn));

And pg_get_wal_stats woule be like:

SELECT * FROM pg_walinpect_stat(pg_walinspect_parse(raw))
FROM (SELECT * FROM pg_walinspect_get_raw(start_lsn, end_lsn)));

Imagine pg_get_raw_wal_record function feeding raw WAL record to an
external tool/extension that understands the WAL. Apart from this, I
don't have a concrete reason either. I'm open to removing this
function as well and adding it along with the raw WAL parsing function
in future.

I haven't thought about the raw WAL parsing functions for now. In
fact, there are many functions we can add to pg_walinspect - functions
with wait mode for future WAL, WAL parsing, function to return all the
WAL record info/stats given a WAL file name, functions to return WAL
info/stats from historic timelines as well, function to see if the
given WAL file is valid and so on. We can park these functions for
future versions of pg_walinspect once the extension itself with basic
yet-useful-and-effective functions gets in. I will make a note of
these functions and will work in future based on how pg_walinspect
gets received by the users and community out there.

Regards,
Bharath Rupireddy.

#52Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Bharath Rupireddy (#44)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Some comments on pg_walinspect-docc.patch this time:

+   <varlistentry>
+    <term>
+     <function>pg_get_wal_record_info(in_lsn pg_lsn, lsn OUT pg_lsn,
prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length
OUT int4, total_length OUT int4, description OUT text, block_ref OUT
text, data OUT bytea, data_len OUT int4)</function>
+    </term>

You may shorten this by mentioning just the function input parameters
and specify "returns record" like shown below. So no need to specify
all the OUT params.

pg_get_wal_record_info(in_lsn pg_lsn) returns record.

Please check the documentation for other functions for reference.

==

+    <term>
+     <function>pg_get_wal_records_info(start_lsn pg_lsn, end_lsn
pg_lsn DEFAULT NULL, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid,
resource_manager OUT text, length OUT int4, total_length OUT int4,
description OUT text, block_ref OUT text, data OUT bytea, data_len OUT
int4)</function>
+    </term>

Same comment applies here as well. In the return type you can just
mention - "returns setof record" like shown below:

pg_get_wal_records_info(start_lsn pg_lsn, end_lsn pg_lsn) returns setof records.

You may also check for such optimizations at other places. I might
have missed some.

==

+<screen>
+postgres=# select prev_lsn, xid, resource_manager, length,
total_length, block_ref from pg_get_wal_records_info('0/158A7F0',
'0/1591400');
+ prev_lsn  | xid | resource_manager | length | total_length |
                       block_ref
+-----------+-----+------------------+--------+--------------+--------------------------------------------------------------------------
+ 0/158A7B8 | 735 | Heap             |     54 |         7838 | blkref
#0: rel 1663/5/2619 blk 18 (FPW); hole: offset: 88, length: 408
+ 0/158A7F0 | 735 | Btree            |     53 |         8133 | blkref
#0: rel 1663/5/2696 blk 1 (FPW); hole: offset: 1632, length: 112
+ 0/158C6A8 | 735 | Heap             |     53 |          873 | blkref
#0: rel 1663/5/1259 blk 0 (FPW); hole: offset: 212, length: 7372

Instead of specifying column names in the targetlist I think it's
better to use "*" so that it will display all the output columns. Also
you may shorten the gap between start and end lsn to reduce the output
size.

==

Any reason for not specifying author name in the .sgml file. Do you
want me to add my name to the author? :)

<para>
Ashutosh Sharma <email>ashu.coek88@gmail.com</email>
</para>
</sect2>

--
With Regards,
Ashutosh Sharma.

On Thu, Mar 10, 2022 at 10:15 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Show quoted text

On Thu, Mar 10, 2022 at 1:52 PM Jeff Davis <pgsql@j-davis.com> wrote:

On Wed, 2022-03-02 at 22:37 +0530, Bharath Rupireddy wrote:

Attaching v6 patch set with above review comments addressed. Please
review it further.

Thanks Jeff for reviewing it. I've posted the latest v7 patch-set
upthread [1] which is having more simple-yet-useful-and-effective
functions.

* Don't issue WARNINGs or other messages for ordinary situations, like
when pg_get_wal_records_info() hits the end of WAL.

v7 patch-set [1] has no warnings, but the functions will error out if
future LSN is specified.

* It feels like the APIs that allow waiting for the end of WAL are
slightly off. Can't you just do pg_get_wal_records_info(start_lsn,
least(pg_current_wal_flush_lsn(), end_lsn)) if you want the non-waiting
behavior? Try to make the API more orthogonal, where a few basic
functions can be combined to give you everything you need, rather than
specifying extra parameters and issuing WARNINGs. I

v7 patch-set [1] onwards waiting mode has been removed for all of the
functions, again to keep things simple-yet-useful-and-effective.
However, we can always add new pg_walinspect functions that wait for
future WAL in the next versions once basic stuff gets committed and if
many users ask for it.

* In the docs, include some example output. I don't see any output in
the tests, which makes sense because it's mostly non-deterministic, but
it would be helpful to see sample output of at least
pg_get_wal_records_info().

+1. Added for pg_get_wal_records_info and pg_get_wal_stats.

* Is pg_get_wal_stats() even necessary, or can you get the same
information with a query over pg_get_wal_records_info()? For instance,
if you want to group by transaction ID rather than rmgr, then
pg_get_wal_stats() is useless.

Yes, you are right pg_get_wal_stats provides WAL stats per resource
manager which is similar to pg_waldump with --start, --end and --stats
option. It provides more information than pg_get_wal_records_info and
is a good way of getting stats than adding more columns to
pg_get_wal_records_info, calculating percentage in sql and having
group by clause. IMO, pg_get_wal_stats is more readable and useful.

* Would be nice to have a pg_wal_file_is_valid() or similar, which
would test that it exists, and the header matches the filename (e.g. if
it was recycled but not used, that would count as invalid). I think
pg_get_first_valid_wal_record_lsn() would make some cases look invalid
even if the file is valid -- for example, if a wal record spans many
wal segments, the segments might look invalid because they contain no
complete records, but the file itself is still valid and contains valid
wal data.

Actually I haven't tried testing a single WAL record spanning many WAL
files yet(I'm happy to try it if someone suggests such a use-case). In
that case too I assume pg_get_first_valid_wal_record_lsn() shouldn't
have a problem because it just gives the next valid LSN and it's
previous LSN using existing WAL reader API XLogFindNextRecord(). It
opens up the WAL file segments using (some dots to connect -
page_read/read_local_xlog_page, WALRead,
segment_open/wal_segment_open). Thoughts?

I don't think it's necessary to have a function pg_wal_file_is_valid()
that given a WAL file name as input checks whether a WAL file exists
or not, probably not in the core (xlogfuncs.c) too. These kinds of
functions can open up challenges in terms of user input validation and
may cause unnecessary problems, please see some related discussion
[2].

* Is there a reason you didn't include the timeline ID in
pg_get_wal_records_info()?

I'm right now allowing the functions to read WAL from the current
server's timeline which I have mentioned in the docs. The server's
current timeline is available via pg_control_checkpoint()'s
timeline_id. So, having timeline_id as a column doesn't make sense.
Again this is to keep things simple-yet-useful-and-effective. However,
we can add new pg_walinspect functions to read WAL from historic as
well as current timelines in the next versions once basic stuff gets
committed and if many users ask for it.

+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>

* Can we mark this extension 'trusted'? I'm not 100% clear on the
standards for that marker, but it seems reasonable for a database owner
with the right privileges might want to install it.

'trusted' extensions concept is added by commit 50fc694 [3]. Since
pg_walinspect deals with WAL, we strictly want to control who creates
and can execute functions exposed by it, so I don't know if 'trusted'
is a good idea here. Also, pageinspect isn't a 'trusted' extension.

* pg_get_raw_wal_record() seems too powerful for pg_monitor. Maybe that
function should require pg_read_server_files? Or at least
pg_read_all_data?

pg_read_all_data may not be the right choice, but pg_read_server_files
is. However, does it sound good if some functions are allowed to be
executed by users with a pg_monitor role and others
pg_get_raw_wal_record by users with pg_read_server_files? Since the
extension itself can be created by superusers, isn't the
pg_get_raw_wal_record sort of safe with pg_mointor itself?

If hackers don't agree, I'm happy to grant execution on
pg_get_raw_wal_record() to the pg_read_server_files role.

Attaching the v8 patch-set resolving above comments and some tests for
checking function permissions. Please review it further.

[1] /messages/by-id/CALj2ACWtToUQ5hCCBJP+mKeVUcN-g7cMb9XvhAcicPxUDsdcKg@mail.gmail.com
[2] /messages/by-id/CA+TgmobYrTgMEF0SV+yDYyCCh44DAGjZVs7BYGrD8xD3vwNjHA@mail.gmail.com
[3] commit 50fc694e43742ce3d04a5e9f708432cb022c5f0d
Author: Tom Lane <tgl@sss.pgh.pa.us>
Date: Wed Jan 29 18:42:43 2020 -0500

Invent "trusted" extensions, and remove the pg_pltemplate catalog.

Regards,
Bharath Rupireddy.

#53Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Kyotaro Horiguchi (#48)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Mar 11, 2022 at 8:22 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

- The difference between pg_get_wal_record_info and _records_ other than
- the number of argument is the former accepts incorrect LSNs.

The discussion is somewhat confused after some twists and turns.. It
should be something like the following.

pg_get_wal_record_info and pg_get_wal_records_info are almost same
since the latter can show a single record. However it is a bit
annoying to do that. Since, other than it doens't accept same LSNs for
start and end, it doesn't show a record when there' no record in the
specfied LSN range. But I don't think there's no usefulness of the
behavior.

I would like to reassert the usability of pg_get_wal_record_info and
pg_get_wal_records_info:

pg_get_wal_record_info(lsn):
if lsn is invalid i.e. '0/0' - throws an error
if lsn is future lsn - throws an error
if lsn looks okay, it figures out the next available valid WAL record
and returns info about that

pg_get_wal_records_info(start_lsn, end_lsn default null) -> if start
and end lsns are provided no end_lsn would give the WAL records info
till the end of WAL,
if start_lsn is invalid i.e. '0/0' - throws an error
if start_lsn is future lsn - throws an error
if end_lsn isn't provided by the user - calculates the end_lsn as
server's current flush lsn
if end_lsn is provided by the user - throws an error if it's future LSN
if start_lsn and end_lsn look okay, it returns info about all WAL
records from the next available valid WAL record of start_lsn until
end_lsn

So, both pg_get_wal_record_info and pg_get_wal_records_info are necessary IMHO.

Coming to the behaviour when input lsn is '0/1000000', it's an issue
with XLogSegmentOffset(lsn, wal_segment_size) != 0 check, which I will
fix in the next version.

if (*first_record != lsn && XLogSegmentOffset(lsn, wal_segment_size) != 0)
ereport(WARNING,
(errmsg_plural("first record is after %X/%X, at %X/%X,
skipping over %u byte",
"first record is after %X/%X, at %X/%X,
skipping over %u bytes",
(*first_record - lsn),
LSN_FORMAT_ARGS(lsn),
LSN_FORMAT_ARGS(*first_record),
(uint32) (*first_record - lsn))));

Regards,
Bharath Rupireddy.

#54Robert Haas
robertmhaas@gmail.com
In reply to: Kyotaro Horiguchi (#47)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Mar 10, 2022 at 9:38 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

It seems to me too rigorous that pg_get_wal_records_info/stats()
reject future LSNs as end-LSN and I think WARNING or INFO and stop at
the real end-of-WAL is more kind to users. I think the same with the
restriction that start and end LSN are required to be different.

In his review just yesterday, Jeff suggested this: "Don't issue
WARNINGs or other messages for ordinary situations, like when
pg_get_wal_records_info() hits the end of WAL." I think he's entirely
right, and I don't think any patch that does otherwise should get
committed. It is worth remembering that the results of queries are
often examined by something other than a human being sitting at a psql
terminal. Any tool that uses this is going to want to understand what
happened from the result set, not by parsing strings that may show up
inside warning messages.

I think that the right answer here is to have a function that returns
one row per record parsed, and each row should also include the start
and end LSN of the record. If for some reason the WAL records return
start after the specified start LSN (e.g. because we skip over a page
header) or end before the specified end LSN (e.g. because we reach
end-of-WAL) the user can figure it out from looking at the LSNs in the
output rows and comparing them to the LSNs provided as input.

--
Robert Haas
EDB: http://www.enterprisedb.com

#55Jeff Davis
pgsql@j-davis.com
In reply to: Stephen Frost (#46)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, 2022-03-10 at 15:54 -0500, Stephen Frost wrote:

The standard is basically that all of the functions it brings are
written to enforce the PG privilege system and you aren't able to use
the extension to bypass those privileges. In some cases that means
that

Every extension should follow that standard, right? If it doesn't (e.g.
creating dangerous functions and granting them to public), then even
superuser should not install it.

the C-language functions installed have if(!superuser) ereport()
calls

I'm curious why not rely on the grant system where possible? I thought
we were trying to get away from explicit superuser checks.

I've not looked back on this thread, but I'd expect pg_walinspect to
need those superuser checks and with those it *could* be marked as
trusted, but that again brings into question how useful it is to mark
it
thusly.

As long as any functions are safely accessible to public or a
predefined role, there is some utility for the 'trusted' marker.

As this patch is currently written, pg_monitor has access these
functions, though I don't think that's the right privilege level at
least for pg_get_raw_wal_record().

I certainly don't think we should allow either database owners or
regular users on a system the ability to access the WAL traffic of
the
entire system.

Agreed. That was not what I intended by asking if it should be marked
'trusted'. The marker only allows the non-superuser to run the CREATE
EXTENSION command; it's up to the extension script to decide whether
any non-superusers can do anything at all with the extension.

More forcefully- we should *not* be throwing more access
rights towards $owners in general and should be thinking about how we
can allow admins, providers, whomever, the ability to control what
rights users are given. If they're all lumped under 'owner' then
there's no way for people to provide granular access to just those
things they wish and intend to.

Not sure I understand, but that sounds like a larger discussion.

Regards,
Jeff Davis

#56Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Ashutosh Sharma (#52)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Mar 11, 2022 at 7:53 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

Some comments on pg_walinspect-docc.patch this time:

+   <varlistentry>
+    <term>
+     <function>pg_get_wal_record_info(in_lsn pg_lsn, lsn OUT pg_lsn,
prev_lsn OUT pg_lsn, xid OUT xid, resource_manager OUT text, length
OUT int4, total_length OUT int4, description OUT text, block_ref OUT
text, data OUT bytea, data_len OUT int4)</function>
+    </term>

You may shorten this by mentioning just the function input parameters
and specify "returns record" like shown below. So no need to specify
all the OUT params.

pg_get_wal_record_info(in_lsn pg_lsn) returns record.

Please check the documentation for other functions for reference.

==

+    <term>
+     <function>pg_get_wal_records_info(start_lsn pg_lsn, end_lsn
pg_lsn DEFAULT NULL, lsn OUT pg_lsn, prev_lsn OUT pg_lsn, xid OUT xid,
resource_manager OUT text, length OUT int4, total_length OUT int4,
description OUT text, block_ref OUT text, data OUT bytea, data_len OUT
int4)</function>
+    </term>

Same comment applies here as well. In the return type you can just
mention - "returns setof record" like shown below:

pg_get_wal_records_info(start_lsn pg_lsn, end_lsn pg_lsn) returns setof records.

You may also check for such optimizations at other places. I might
have missed some.

I used the way verify_heapam shows the columns as it looks good IMO
and we can't show sample outputs for all of the functions in the
documentation.

==

+<screen>
+postgres=# select prev_lsn, xid, resource_manager, length,
total_length, block_ref from pg_get_wal_records_info('0/158A7F0',
'0/1591400');
+ prev_lsn  | xid | resource_manager | length | total_length |
block_ref
+-----------+-----+------------------+--------+--------------+--------------------------------------------------------------------------
+ 0/158A7B8 | 735 | Heap             |     54 |         7838 | blkref
#0: rel 1663/5/2619 blk 18 (FPW); hole: offset: 88, length: 408
+ 0/158A7F0 | 735 | Btree            |     53 |         8133 | blkref
#0: rel 1663/5/2696 blk 1 (FPW); hole: offset: 1632, length: 112
+ 0/158C6A8 | 735 | Heap             |     53 |          873 | blkref
#0: rel 1663/5/1259 blk 0 (FPW); hole: offset: 212, length: 7372

Instead of specifying column names in the targetlist I think it's
better to use "*" so that it will display all the output columns. Also
you may shorten the gap between start and end lsn to reduce the output
size.

All columns are giving huge output, especially because of data and
description columns hence I'm not showing them in the sample output.

==

Any reason for not specifying author name in the .sgml file. Do you
want me to add my name to the author? :)

Haha. Thanks. I will add in the v9 patch set which I will post in a while.

Regards,
Bharath Rupireddy.

#57Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Robert Haas (#54)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Sat, Mar 12, 2022 at 2:09 AM Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Mar 10, 2022 at 9:38 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

It seems to me too rigorous that pg_get_wal_records_info/stats()
reject future LSNs as end-LSN and I think WARNING or INFO and stop at
the real end-of-WAL is more kind to users. I think the same with the
restriction that start and end LSN are required to be different.

In his review just yesterday, Jeff suggested this: "Don't issue
WARNINGs or other messages for ordinary situations, like when
pg_get_wal_records_info() hits the end of WAL." I think he's entirely
right, and I don't think any patch that does otherwise should get
committed. It is worth remembering that the results of queries are
often examined by something other than a human being sitting at a psql
terminal. Any tool that uses this is going to want to understand what
happened from the result set, not by parsing strings that may show up
inside warning messages.

I think that the right answer here is to have a function that returns
one row per record parsed, and each row should also include the start
and end LSN of the record. If for some reason the WAL records return
start after the specified start LSN (e.g. because we skip over a page
header) or end before the specified end LSN (e.g. because we reach
end-of-WAL) the user can figure it out from looking at the LSNs in the
output rows and comparing them to the LSNs provided as input.

Thanks Robert. I've removed the WARNING part and added end_lsn as suggested.

Thanks Kyotaro-san, Ashutosh and Jeff for your review. I tried to
address your review comments, if not all, but many.

Here's v9 patch-set please review it further.

Regards,
Bharath Rupireddy.

Attachments:

v9-0001-pg_walinspect.patchapplication/octet-stream; name=v9-0001-pg_walinspect.patchDownload
From e2b5bfd63508313f05f77a0bf157789b0585cf95 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sat, 12 Mar 2022 10:26:53 +0000
Subject: [PATCH v9] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql |  95 ++++
 contrib/pg_walinspect/pg_walinspect.c        | 562 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  14 +-
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/common/relpath.c                         |  18 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/common/relpath.h                 |   1 +
 13 files changed, 726 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index e3e221308b..705c6fc36b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -40,6 +40,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..eb5c577c47
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,95 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT record_length int4,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_length int8,
+    OUT record_length_percentage float4,
+    OUT fpi_length int8,
+    OUT fpi_length_percentage float4
+    )
+RETURNS SETOF record AS $$
+SELECT resource_manager,
+    count(*) AS cnt,
+    CASE WHEN count(*) > 0 THEN (count(*) * 100 / sum(count(*)) OVER total)::numeric(5,2) ELSE 0 END AS "count_%",
+    sum(record_length) AS trecl,
+    CASE WHEN sum(record_length) > 0 THEN (sum(record_length) * 100 / sum(sum(record_length)) OVER total)::numeric(5,2) ELSE 0 END AS "trecl_%",
+    sum(fpi_length) AS tfpil,
+    CASE WHEN sum(fpi_length) > 0 THEN (sum(fpi_length) * 100 / sum(sum(fpi_length)) OVER total)::numeric(5,2) ELSE 0 END AS "tfpil_%"
+FROM pg_get_wal_records_info(start_lsn, end_lsn)
+GROUP BY resource_manager
+WINDOW total AS ();
+$$ LANGUAGE SQL CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..ab240b46e4
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,562 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+
+static XLogRecPtr ValidateInputLSN(XLogRecPtr lsn);
+static XLogRecPtr ValidateStartAndEndLSNs(XLogRecPtr start_lsn,
+										  XLogRecPtr end_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void GetWALRecordsInfoInternal(FunctionCallInfo fcinfo,
+									  XLogRecPtr start_lsn,
+									  XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Validate given LSN and return the LSN up to which the server has WAL.
+ */
+static XLogRecPtr
+ValidateInputLSN(XLogRecPtr lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate input WAL LSN. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(curr_lsn));
+
+	if (lsn >= curr_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	return curr_lsn;
+}
+
+/*
+ * Validate given start LSN and end LSN, return the new end LSN in case user
+ * hasn't specified one.
+ */
+static XLogRecPtr
+ValidateStartAndEndLSNs(XLogRecPtr start_lsn, XLogRecPtr end_lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate WAL start LSN. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL start LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	if (start_lsn >= curr_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	/*
+	 * If end_lsn is specified, let's ensure that it's not a future LSN i.e.
+	 * something the database system doesn't know about.
+	 */
+	if (!XLogRecPtrIsInvalid(end_lsn) &&
+		(end_lsn >= curr_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	/*
+	 * When end_lsn is not specified let's read up to the last WAL position
+	 * known to be on the server.
+	 */
+	if (XLogRecPtrIsInvalid(end_lsn))
+		end_lsn = curr_lsn;
+
+	Assert(!XLogRecPtrIsInvalid(end_lsn));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	return end_lsn;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get raw WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 5
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	*raw_record;
+	uint32	rec_len;
+	char	*raw_record_data;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(xlogreader->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(xlogreader));
+	values[i++] = UInt32GetDatum(rec_len);
+	values[i++] = PointerGetDatum(raw_record);
+
+	XLogReaderFree(xlogreader);
+
+	Assert(i == PG_GET_RAW_WAL_RECORD_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_RAW_WAL_RECORD_COLS
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int	block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	initStringInfo(&rec_desc);
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+	pfree(temp.data);
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		else
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			/* Calculate the amount of FPI data in the record. */
+			fpi_len += record->blocks[block_id].bimg_len;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	main_data_len = XLogRecGetDataLen(record);
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = PointerGetDatum(data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+					  PG_GET_WAL_RECORD_INFO_COLS);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+						  PG_GET_WAL_RECORDS_INFO_COLS);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future end WAL LSN i.e. WAL LSN the
+ * database system doesn't know about is specified.
+ *
+ * This function will figure out the end LSN if it's not specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	end_lsn = ValidateStartAndEndLSNs(start_lsn, end_lsn);
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index b7c06da255..1b072f9c11 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -959,13 +959,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1083,6 +1076,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index f128050b4e..aac41a0793 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 09f6464331..3e644372f9 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 0e94833129..8468732d76 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 477f0efe26..d62d6ce7f8 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v9-0001-pg_walinspect-tests.patchapplication/octet-stream; name=v9-0001-pg_walinspect-tests.patchDownload
From 111d4efaf18bb33a315925fa468a7dfb055568c6 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sat, 12 Mar 2022 10:14:59 +0000
Subject: [PATCH v9] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 161 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 119 +++++++++++++
 2 files changed, 280 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..4c3b422444
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,161 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 10);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 10);
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+CONTEXT:  SQL function "pg_get_wal_stats" statement 1
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+GRANT pg_monitor TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..14a1c6b32f
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,119 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 10);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 10);
+
+-- ===================================================================
+-- tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+-- ===================================================================
+-- tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+-- ===================================================================
+-- tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+GRANT pg_monitor TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v9-0001-pg_walinspect-docs.patchapplication/octet-stream; name=v9-0001-pg_walinspect-docs.patchDownload
From fe48b5c02a15eec522e8dfa0bb8ee1ba00d86d52 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sat, 12 Mar 2022 11:33:22 +0000
Subject: [PATCH v9] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 184 +++++++++++++++++++++++++++++++++
 3 files changed, 186 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index be9711c6f2..19614a42e1 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -130,6 +130,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index 328cd1f378..a2e8fd4a08 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -146,6 +146,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..c49bea9ca0
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,184 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_monitor</literal> role. Access may be granted to others using
+  <command>GRANT</command>.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_raw_wal_record(in_lsn pg_lsn,
+                            start_lsn OUT pg_lsn,
+                            end_lsn OUT pg_lsn,
+                            record_length OUT int4,
+                            record OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. This function emits an error if
+      a future (the LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text,
+                             data_length OUT int4,
+                             data OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of the given LSN. This function emits an
+      error if a future (the LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text,
+                              data_length OUT int4,
+                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between <replaceable>start_lsn</replaceable>
+      and <replaceable>end_lsn</replaceable>. Returns one row per each valid
+      WAL record. This function figures out the <replaceable>end_lsn</replaceable>
+      if it's not specified, that means, it returns information up to the end
+      of WAL. Default value of <replaceable>end_lsn</replaceable> is
+      <literal>NULL</literal>. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>start_lsn</replaceable>
+      or <replaceable>end_lsn</replaceable> is specified. For example:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_length, fpi_length, block_ref, data_length from pg_get_wal_records_info('0/1602790', '0/16038D0');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager | record_length | fpi_length |                                block_ref                                 | data_length 
+-----------+-----------+-----------+-----+------------------+---------------+------------+--------------------------------------------------------------------------+-------------
+ 0/1602790 | 0/16027C7 | 0/1602758 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/16027C8 | 0/160283F | 0/1602790 |   0 | XLOG             |           114 |          0 |                                                                          |          88
+ 0/1602840 | 0/1602877 | 0/16027C8 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/1602878 | 0/16028AF | 0/1602840 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/16028B0 | 0/1602927 | 0/1602878 |   0 | XLOG             |           114 |          0 |                                                                          |          88
+ 0/1602928 | 0/16037AF | 0/16028B0 | 751 | Heap             |          3714 |       3660 | blkref #0: rel 1663/5/24628 blk 0 (FPW); hole: offset: 428, length: 4532 |           3
+ 0/16037B0 | 0/16037EF | 0/1602928 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/16037F0 | 0/160382F | 0/16037B0 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/1603830 | 0/160386F | 0/16037F0 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/1603870 | 0/16038A7 | 0/1603830 |   0 | Standby          |            54 |          0 |                                                                          |          28
+ 0/16038A8 | 0/16038CF | 0/1603870 | 751 | Transaction      |            34 |          0 |                                                                          |           8
+(11 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       resource_manager OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between <replaceable>start_lsn</replaceable>
+      and <replaceable>end_lsn</replaceable>. Returns one row per each <replaceable>resource_manager</replaceable>
+      type. This function figures out the <replaceable>end_lsn</replaceable> if
+      it's not specified, that means, it returns information up to the end of
+      WAL. Default value of <replaceable>end_lsn</replaceable> is
+      <literal>NULL</literal>. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>start_lsn</replaceable>
+      or <replaceable>end_lsn</replaceable> is specified. For example:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/15E80C0', '0/1600000');
+ resource_manager | count | count_percentage | record_length | record_length_percentage | fpi_length | fpi_length_percentage 
+------------------+-------+------------------+---------------+--------------------------+------------+-----------------------
+ Storage          |     1 |              0.7 |            42 |                     0.04 |          0 |                     0
+ Btree            |    27 |            18.88 |         65116 |                    66.98 |      63456 |                 72.45
+ Heap             |   101 |            70.63 |          7697 |                     7.92 |       1596 |                  1.82
+ Transaction      |     1 |              0.7 |           405 |                     0.42 |          0 |                     0
+ Heap2            |     6 |              4.2 |         23573 |                    24.25 |      22532 |                 25.73
+ Standby          |     5 |              3.5 |           242 |                     0.25 |          0 |                     0
+ XLOG             |     2 |              1.4 |           144 |                     0.15 |          0 |                     0
+(7 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#58Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Robert Haas (#54)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

At Fri, 11 Mar 2022 15:39:13 -0500, Robert Haas <robertmhaas@gmail.com> wrote in

On Thu, Mar 10, 2022 at 9:38 PM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

It seems to me too rigorous that pg_get_wal_records_info/stats()
reject future LSNs as end-LSN and I think WARNING or INFO and stop at
the real end-of-WAL is more kind to users. I think the same with the
restriction that start and end LSN are required to be different.

In his review just yesterday, Jeff suggested this: "Don't issue
WARNINGs or other messages for ordinary situations, like when
pg_get_wal_records_info() hits the end of WAL." I think he's entirely
right, and I don't think any patch that does otherwise should get

It depends on what we think is the "ordinary" here. If we don't
expect that specified LSN range is not filled-out, the case above is
ordinary and no need for any WARNING nor INFO. I'm fine with that
definition here.

committed. It is worth remembering that the results of queries are
often examined by something other than a human being sitting at a psql
terminal. Any tool that uses this is going to want to understand what
happened from the result set, not by parsing strings that may show up
inside warning messages.

Right. I don't think it is right that WARNING is required to evaluate
the result. And I think that the WARNING like 'reached end-of-wal
before end LSN' is a kind that is not required in evaluation of the
result. Since each WAL row contains at least start LSN.

I think that the right answer here is to have a function that returns
one row per record parsed, and each row should also include the start
and end LSN of the record. If for some reason the WAL records return
start after the specified start LSN (e.g. because we skip over a page
header) or end before the specified end LSN (e.g. because we reach
end-of-WAL) the user can figure it out from looking at the LSNs in the
output rows and comparing them to the LSNs provided as input.

I agree with you here.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#59Stephen Frost
sfrost@snowman.net
In reply to: Jeff Davis (#55)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Greetings,

* Jeff Davis (pgsql@j-davis.com) wrote:

On Thu, 2022-03-10 at 15:54 -0500, Stephen Frost wrote:

The standard is basically that all of the functions it brings are
written to enforce the PG privilege system and you aren't able to use
the extension to bypass those privileges. In some cases that means
that

Every extension should follow that standard, right? If it doesn't (e.g.
creating dangerous functions and granting them to public), then even
superuser should not install it.

Every extension that's intended to be installed on a multi-user PG
system where the admin cares about security in the database, sure. I
disagree that this applies universally to every system or every
extension. Those are standards that modules we distribute in contrib
should meet but I don't know that we necessarily have to have them for,
say, modules in test.

the C-language functions installed have if(!superuser) ereport()
calls

I'm curious why not rely on the grant system where possible? I thought
we were trying to get away from explicit superuser checks.

We don't yet have capabilities for everything. I agree that we should
be getting away from explicit superuser checks and explained below how
we might be able to in this case.

I've not looked back on this thread, but I'd expect pg_walinspect to
need those superuser checks and with those it *could* be marked as
trusted, but that again brings into question how useful it is to mark
it
thusly.

As long as any functions are safely accessible to public or a
predefined role, there is some utility for the 'trusted' marker.

I'm not sure that I agree, though I'm also not sure that it's a useful
thing to debate. Still, if all of the functions in a particular
extension have explicit if (!superuser) ereport() checks in them, then
while installing it is 'safe' and it could be marked as 'trusted'
there's very little point in doing so as the only person who can get
anything useful from those functions is a superuser, and a superuser can
install non-trusted extensions anyway. How is it useful to mark such an
extension as 'trusted'?

As this patch is currently written, pg_monitor has access these
functions, though I don't think that's the right privilege level at
least for pg_get_raw_wal_record().

Yeah, I agree that pg_monitor isn't the right thing for such a function
to be checking.

I certainly don't think we should allow either database owners or
regular users on a system the ability to access the WAL traffic of
the
entire system.

Agreed. That was not what I intended by asking if it should be marked
'trusted'. The marker only allows the non-superuser to run the CREATE
EXTENSION command; it's up to the extension script to decide whether
any non-superusers can do anything at all with the extension.

Sure.

More forcefully- we should *not* be throwing more access
rights towards $owners in general and should be thinking about how we
can allow admins, providers, whomever, the ability to control what
rights users are given. If they're all lumped under 'owner' then
there's no way for people to provide granular access to just those
things they wish and intend to.

Not sure I understand, but that sounds like a larger discussion.

The point I was trying to make is that it's better to move in the
direction of things like pg_read_all_data rather than just declaring
that the owner of a database can read all of the tables in that
database, as an example. Maybe we want to implicitly have such
privilege for the owner of the database too, but we should first make it
something that's able to be GRANT'd out to non-owners so that it's not
required that all of those privileges be given out together at once.

Note that to be considered an 'owner' of an object you have to be a
member of the role that owns the object, which means that said role is
necessarily able to also become the owning role too. Lumping lots of
privileges together- all the rights that being an 'owner' of the object
conveys, plus the ability to also become that role directly and do
things as that role, works actively against the general idea of 'least
privilege'.

Thanks,

Stephen

#60Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Stephen Frost (#59)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Mon, Mar 14, 2022 at 8:25 PM Stephen Frost <sfrost@snowman.net> wrote:

As this patch is currently written, pg_monitor has access these
functions, though I don't think that's the right privilege level at
least for pg_get_raw_wal_record().

Yeah, I agree that pg_monitor isn't the right thing for such a function
to be checking.

On Thu, Mar 10, 2022 at 1:52 PM Jeff Davis <pgsql@j-davis.com> wrote:

* pg_get_raw_wal_record() seems too powerful for pg_monitor. Maybe that
function should require pg_read_server_files? Or at least
pg_read_all_data?

The v9 patch set posted at [1]/messages/by-id/CALj2ACVRH-z8mZLyFkpLvY4qRhxQCqU_BLkFTtwt+TPZNhfEVg@mail.gmail.com grants execution on
pg_get_raw_wal_record() to the pg_monitor role.

pg_read_all_data may not be the right choice, but pg_read_server_files
is as these functions do read the WAL files on the server. If okay,
I'm happy to grant execution on pg_get_raw_wal_record() to the
pg_read_server_files role.

Thoughts?

[1]: /messages/by-id/CALj2ACVRH-z8mZLyFkpLvY4qRhxQCqU_BLkFTtwt+TPZNhfEVg@mail.gmail.com

Regards,
Bharath Rupireddy.

#61Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Bharath Rupireddy (#60)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Tue, Mar 15, 2022 at 7:21 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

On Mon, Mar 14, 2022 at 8:25 PM Stephen Frost <sfrost@snowman.net> wrote:

As this patch is currently written, pg_monitor has access these
functions, though I don't think that's the right privilege level at
least for pg_get_raw_wal_record().

Yeah, I agree that pg_monitor isn't the right thing for such a function
to be checking.

On Thu, Mar 10, 2022 at 1:52 PM Jeff Davis <pgsql@j-davis.com> wrote:

* pg_get_raw_wal_record() seems too powerful for pg_monitor. Maybe that
function should require pg_read_server_files? Or at least
pg_read_all_data?

The v9 patch set posted at [1] grants execution on
pg_get_raw_wal_record() to the pg_monitor role.

pg_read_all_data may not be the right choice, but pg_read_server_files
is as these functions do read the WAL files on the server. If okay,
I'm happy to grant execution on pg_get_raw_wal_record() to the
pg_read_server_files role.

Thoughts?

[1] /messages/by-id/CALj2ACVRH-z8mZLyFkpLvY4qRhxQCqU_BLkFTtwt+TPZNhfEVg@mail.gmail.com

Attaching v10 patch set which allows pg_get_raw_wal_record to be
executed by either superuser or users with pg_read_server_files role,
no other change from v9 patch set.

Please review it further.

Regards,
Bharath Rupireddy.

Attachments:

v10-0001-pg_walinspect.patchapplication/octet-stream; name=v10-0001-pg_walinspect.patchDownload
From 38a2e9b44b0113059050b82d52450300915d1e3e Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 16 Mar 2022 07:18:57 +0000
Subject: [PATCH v10] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql |  95 ++++
 contrib/pg_walinspect/pg_walinspect.c        | 562 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  14 +-
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/common/relpath.c                         |  18 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/common/relpath.h                 |   1 +
 13 files changed, 726 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 332b486ecc..bbf220407b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -41,6 +41,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..14d7642350
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,95 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT record_length int4,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_length int8,
+    OUT record_length_percentage float4,
+    OUT fpi_length int8,
+    OUT fpi_length_percentage float4
+    )
+RETURNS SETOF record AS $$
+SELECT resource_manager,
+    count(*) AS cnt,
+    CASE WHEN count(*) > 0 THEN (count(*) * 100 / sum(count(*)) OVER total)::numeric(5,2) ELSE 0 END AS "count_%",
+    sum(record_length) AS trecl,
+    CASE WHEN sum(record_length) > 0 THEN (sum(record_length) * 100 / sum(sum(record_length)) OVER total)::numeric(5,2) ELSE 0 END AS "trecl_%",
+    sum(fpi_length) AS tfpil,
+    CASE WHEN sum(fpi_length) > 0 THEN (sum(fpi_length) * 100 / sum(sum(fpi_length)) OVER total)::numeric(5,2) ELSE 0 END AS "tfpil_%"
+FROM pg_get_wal_records_info(start_lsn, end_lsn)
+GROUP BY resource_manager
+WINDOW total AS ();
+$$ LANGUAGE SQL CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..ab240b46e4
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,562 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+
+static XLogRecPtr ValidateInputLSN(XLogRecPtr lsn);
+static XLogRecPtr ValidateStartAndEndLSNs(XLogRecPtr start_lsn,
+										  XLogRecPtr end_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void GetWALRecordsInfoInternal(FunctionCallInfo fcinfo,
+									  XLogRecPtr start_lsn,
+									  XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Validate given LSN and return the LSN up to which the server has WAL.
+ */
+static XLogRecPtr
+ValidateInputLSN(XLogRecPtr lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate input WAL LSN. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(curr_lsn));
+
+	if (lsn >= curr_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	return curr_lsn;
+}
+
+/*
+ * Validate given start LSN and end LSN, return the new end LSN in case user
+ * hasn't specified one.
+ */
+static XLogRecPtr
+ValidateStartAndEndLSNs(XLogRecPtr start_lsn, XLogRecPtr end_lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate WAL start LSN. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL start LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	if (start_lsn >= curr_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	/*
+	 * If end_lsn is specified, let's ensure that it's not a future LSN i.e.
+	 * something the database system doesn't know about.
+	 */
+	if (!XLogRecPtrIsInvalid(end_lsn) &&
+		(end_lsn >= curr_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn))));
+
+	/*
+	 * When end_lsn is not specified let's read up to the last WAL position
+	 * known to be on the server.
+	 */
+	if (XLogRecPtrIsInvalid(end_lsn))
+		end_lsn = curr_lsn;
+
+	Assert(!XLogRecPtrIsInvalid(end_lsn));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	return end_lsn;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get raw WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 5
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	*raw_record;
+	uint32	rec_len;
+	char	*raw_record_data;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(xlogreader->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(xlogreader));
+	values[i++] = UInt32GetDatum(rec_len);
+	values[i++] = PointerGetDatum(raw_record);
+
+	XLogReaderFree(xlogreader);
+
+	Assert(i == PG_GET_RAW_WAL_RECORD_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_RAW_WAL_RECORD_COLS
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int	block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	initStringInfo(&rec_desc);
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+	pfree(temp.data);
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		else
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			/* Calculate the amount of FPI data in the record. */
+			fpi_len += record->blocks[block_id].bimg_len;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	main_data_len = XLogRecGetDataLen(record);
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = PointerGetDatum(data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+					  PG_GET_WAL_RECORD_INFO_COLS);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		/* Exit loop if read up to end_lsn. */
+		if (!XLogRecPtrIsInvalid(end_lsn) &&
+			xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+						  PG_GET_WAL_RECORDS_INFO_COLS);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future end WAL LSN i.e. WAL LSN the
+ * database system doesn't know about is specified.
+ *
+ * This function will figure out the end LSN if it's not specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	end_lsn = ValidateStartAndEndLSNs(start_lsn, end_lsn);
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index b7c06da255..1b072f9c11 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -959,13 +959,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1083,6 +1076,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index f128050b4e..aac41a0793 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 09f6464331..3e644372f9 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 0e94833129..8468732d76 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 477f0efe26..d62d6ce7f8 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v10-0001-pg_walinspect-tests.patchapplication/octet-stream; name=v10-0001-pg_walinspect-tests.patchDownload
From 2d346bd2fce444e2bf0bf9ad69f5f05ec55eae70 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 16 Mar 2022 07:26:55 +0000
Subject: [PATCH v10] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 166 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 128 ++++++++++++++
 2 files changed, 294 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..0b36944838
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,166 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 10);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 10);
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+CONTEXT:  SQL function "pg_get_wal_stats" statement 1
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+-- Function accessible by users with role pg_read_server_files
+GRANT pg_read_server_files TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+-- Functions accessible by users with role pg_monitor
+GRANT pg_monitor TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..96bc3ef7bd
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,128 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 10);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 10);
+
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+-- Function accessible by users with role pg_read_server_files
+
+GRANT pg_read_server_files TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+
+-- Functions accessible by users with role pg_monitor
+
+GRANT pg_monitor TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v10-0001-pg_walinspect-docs.patchapplication/octet-stream; name=v10-0001-pg_walinspect-docs.patchDownload
From afac3bfc209ca6ee236cb9e796395b2bd34ca838 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 16 Mar 2022 07:38:52 +0000
Subject: [PATCH v10] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 202 +++++++++++++++++++++++++++++++++
 3 files changed, 204 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index 1e42ce1a7f..4e7b87a42f 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -131,6 +131,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index fd853af01f..34c19c80f1 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -147,6 +147,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..d47d98b91e
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,202 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_raw_wal_record(in_lsn pg_lsn,
+                            start_lsn OUT pg_lsn,
+                            end_lsn OUT pg_lsn,
+                            record_length OUT int4,
+                            record OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. This function emits an error if
+      a future (the LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_read_server_files</literal> role. Access may be
+       granted by superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text,
+                             data_length OUT int4,
+                             data OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of the given LSN. This function emits an
+      error if a future (the LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text,
+                              data_length OUT int4,
+                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between <replaceable>start_lsn</replaceable>
+      and <replaceable>end_lsn</replaceable>. Returns one row per each valid
+      WAL record. This function figures out the <replaceable>end_lsn</replaceable>
+      if it's not specified, that means, it returns information up to the end
+      of WAL. Default value of <replaceable>end_lsn</replaceable> is
+      <literal>NULL</literal>. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>start_lsn</replaceable>
+      or <replaceable>end_lsn</replaceable> is specified. For example:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_length, fpi_length, block_ref, data_length from pg_get_wal_records_info('0/1602790', '0/16038D0');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager | record_length | fpi_length |                                block_ref                                 | data_length 
+-----------+-----------+-----------+-----+------------------+---------------+------------+--------------------------------------------------------------------------+-------------
+ 0/1602790 | 0/16027C7 | 0/1602758 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/16027C8 | 0/160283F | 0/1602790 |   0 | XLOG             |           114 |          0 |                                                                          |          88
+ 0/1602840 | 0/1602877 | 0/16027C8 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/1602878 | 0/16028AF | 0/1602840 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/16028B0 | 0/1602927 | 0/1602878 |   0 | XLOG             |           114 |          0 |                                                                          |          88
+ 0/1602928 | 0/16037AF | 0/16028B0 | 751 | Heap             |          3714 |       3660 | blkref #0: rel 1663/5/24628 blk 0 (FPW); hole: offset: 428, length: 4532 |           3
+ 0/16037B0 | 0/16037EF | 0/1602928 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/16037F0 | 0/160382F | 0/16037B0 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/1603830 | 0/160386F | 0/16037F0 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/1603870 | 0/16038A7 | 0/1603830 |   0 | Standby          |            54 |          0 |                                                                          |          28
+ 0/16038A8 | 0/16038CF | 0/1603870 | 751 | Transaction      |            34 |          0 |                                                                          |           8
+(11 rows)
+</screen>
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       resource_manager OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between <replaceable>start_lsn</replaceable>
+      and <replaceable>end_lsn</replaceable>. Returns one row per each <replaceable>resource_manager</replaceable>
+      type. This function figures out the <replaceable>end_lsn</replaceable> if
+      it's not specified, that means, it returns information up to the end of
+      WAL. Default value of <replaceable>end_lsn</replaceable> is
+      <literal>NULL</literal>. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>start_lsn</replaceable>
+      or <replaceable>end_lsn</replaceable> is specified. For example:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/15E80C0', '0/1600000');
+ resource_manager | count | count_percentage | record_length | record_length_percentage | fpi_length | fpi_length_percentage 
+------------------+-------+------------------+---------------+--------------------------+------------+-----------------------
+ Storage          |     1 |              0.7 |            42 |                     0.04 |          0 |                     0
+ Btree            |    27 |            18.88 |         65116 |                    66.98 |      63456 |                 72.45
+ Heap             |   101 |            70.63 |          7697 |                     7.92 |       1596 |                  1.82
+ Transaction      |     1 |              0.7 |           405 |                     0.42 |          0 |                     0
+ Heap2            |     6 |              4.2 |         23573 |                    24.25 |      22532 |                 25.73
+ Standby          |     5 |              3.5 |           242 |                     0.25 |          0 |                     0
+ XLOG             |     2 |              1.4 |           144 |                     0.15 |          0 |                     0
+(7 rows)
+</screen>
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#62Stephen Frost
sfrost@snowman.net
In reply to: Bharath Rupireddy (#61)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Greetings,

* Bharath Rupireddy (bharath.rupireddyforpostgres@gmail.com) wrote:

On Tue, Mar 15, 2022 at 7:21 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

On Mon, Mar 14, 2022 at 8:25 PM Stephen Frost <sfrost@snowman.net> wrote:

As this patch is currently written, pg_monitor has access these
functions, though I don't think that's the right privilege level at
least for pg_get_raw_wal_record().

Yeah, I agree that pg_monitor isn't the right thing for such a function
to be checking.

On Thu, Mar 10, 2022 at 1:52 PM Jeff Davis <pgsql@j-davis.com> wrote:

* pg_get_raw_wal_record() seems too powerful for pg_monitor. Maybe that
function should require pg_read_server_files? Or at least
pg_read_all_data?

The v9 patch set posted at [1] grants execution on
pg_get_raw_wal_record() to the pg_monitor role.

pg_read_all_data may not be the right choice, but pg_read_server_files
is as these functions do read the WAL files on the server. If okay,
I'm happy to grant execution on pg_get_raw_wal_record() to the
pg_read_server_files role.

Thoughts?

[1] /messages/by-id/CALj2ACVRH-z8mZLyFkpLvY4qRhxQCqU_BLkFTtwt+TPZNhfEVg@mail.gmail.com

Attaching v10 patch set which allows pg_get_raw_wal_record to be
executed by either superuser or users with pg_read_server_files role,
no other change from v9 patch set.

In a quick look, that seems reasonable to me. If folks want to give out
access to this function individually they're also able to do so, which
is good. Doesn't seem worthwhile to introduce a new predefined role for
this one function.

Thanks,

Stephen

#63Ashutosh Sharma
ashu.coek88@gmail.com
In reply to: Stephen Frost (#62)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

I can see that the pg_get_wal_records_info function shows the details
of the WAL record whose existence is beyond the user specified
stop/end lsn pointer. See below:

ashu@postgres=# select * from pg_get_wal_records_info('0/01000028',
'0/01000029');
-[ RECORD 1 ]----+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
start_lsn | 0/1000028
end_lsn | 0/100009F
prev_lsn | 0/0
xid | 0
resource_manager | XLOG
record_length | 114
fpi_length | 0
description | CHECKPOINT_SHUTDOWN redo 0/1000028; tli 1; prev tli
1; fpw true; xid 0:3; oid 10000; multi 1; offset 0; oldest xid 3 in DB
1; oldest multi 1 in DB 1; oldest/newest commit timestamp xid: 0/0;
oldest running xid 0; shutdown
block_ref |
data_length | 88
data |
\x28000001000000000100000001000000010000000000000003000000000000001027000001000000000000000300000001000000010000000100000072550000a5c4316200000000000000000000000000000000ff7f0000

In this case, the end lsn pointer specified by the user is
'0/01000029'. There is only one WAL record which starts before this
specified end lsn pointer whose start pointer is at 01000028, but that
WAL record ends at 0/100009F which is way beyond the specified end
lsn. So, how come we are able to display the complete WAL record info?
AFAIU, end lsn is the lsn pointer where you need to stop reading the
WAL data. If that is true, then there exists no valid WAL record
between the start and end lsn in this particular case.

--
With Regards,
Ashutosh Sharma.

Show quoted text

On Wed, Mar 16, 2022 at 7:56 PM Stephen Frost <sfrost@snowman.net> wrote:

Greetings,

* Bharath Rupireddy (bharath.rupireddyforpostgres@gmail.com) wrote:

On Tue, Mar 15, 2022 at 7:21 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

On Mon, Mar 14, 2022 at 8:25 PM Stephen Frost <sfrost@snowman.net> wrote:

As this patch is currently written, pg_monitor has access these
functions, though I don't think that's the right privilege level at
least for pg_get_raw_wal_record().

Yeah, I agree that pg_monitor isn't the right thing for such a function
to be checking.

On Thu, Mar 10, 2022 at 1:52 PM Jeff Davis <pgsql@j-davis.com> wrote:

* pg_get_raw_wal_record() seems too powerful for pg_monitor. Maybe that
function should require pg_read_server_files? Or at least
pg_read_all_data?

The v9 patch set posted at [1] grants execution on
pg_get_raw_wal_record() to the pg_monitor role.

pg_read_all_data may not be the right choice, but pg_read_server_files
is as these functions do read the WAL files on the server. If okay,
I'm happy to grant execution on pg_get_raw_wal_record() to the
pg_read_server_files role.

Thoughts?

[1] /messages/by-id/CALj2ACVRH-z8mZLyFkpLvY4qRhxQCqU_BLkFTtwt+TPZNhfEVg@mail.gmail.com

Attaching v10 patch set which allows pg_get_raw_wal_record to be
executed by either superuser or users with pg_read_server_files role,
no other change from v9 patch set.

In a quick look, that seems reasonable to me. If folks want to give out
access to this function individually they're also able to do so, which
is good. Doesn't seem worthwhile to introduce a new predefined role for
this one function.

Thanks,

Stephen

#64Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Ashutosh Sharma (#63)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

At Wed, 16 Mar 2022 20:49:12 +0530, Ashutosh Sharma <ashu.coek88@gmail.com> wrote in

I can see that the pg_get_wal_records_info function shows the details
of the WAL record whose existence is beyond the user specified
stop/end lsn pointer. See below:

ashu@postgres=# select * from pg_get_wal_records_info('0/01000028',
'0/01000029');
-[ RECORD 1 ]----+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
start_lsn | 0/1000028
end_lsn | 0/100009F
prev_lsn | 0/0

...

record_length | 114

...

In this case, the end lsn pointer specified by the user is
'0/01000029'. There is only one WAL record which starts before this
specified end lsn pointer whose start pointer is at 01000028, but that
WAL record ends at 0/100009F which is way beyond the specified end
lsn. So, how come we are able to display the complete WAL record info?
AFAIU, end lsn is the lsn pointer where you need to stop reading the
WAL data. If that is true, then there exists no valid WAL record
between the start and end lsn in this particular case.

You're right considering how pg_waldump behaves. pg_waldump works
almost the way as you described above. The record above actually ends
at 1000099 and pg_waldump shows that record by specifying -s 0/1000028
-e 0/100009a, but not for -e 0/1000099.

# I personally think the current behavior is fine, though..

It still suggests unspecifiable end-LSN..

select * from pg_get_wal_records_info('4/4B28EB68', '4/4C000060');
ERROR: cannot accept future end LSN
DETAIL: Last known WAL LSN on the database system is 4/4C000060.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#65Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Ashutosh Sharma (#63)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Mar 16, 2022 at 8:49 PM Ashutosh Sharma <ashu.coek88@gmail.com> wrote:

I can see that the pg_get_wal_records_info function shows the details
of the WAL record whose existence is beyond the user specified
stop/end lsn pointer. See below:

ashu@postgres=# select * from pg_get_wal_records_info('0/01000028',
'0/01000029');
-[ RECORD 1 ]----+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
start_lsn | 0/1000028
end_lsn | 0/100009F
prev_lsn | 0/0
xid | 0
resource_manager | XLOG
record_length | 114
fpi_length | 0
description | CHECKPOINT_SHUTDOWN redo 0/1000028; tli 1; prev tli
1; fpw true; xid 0:3; oid 10000; multi 1; offset 0; oldest xid 3 in DB
1; oldest multi 1 in DB 1; oldest/newest commit timestamp xid: 0/0;
oldest running xid 0; shutdown
block_ref |
data_length | 88
data |
\x28000001000000000100000001000000010000000000000003000000000000001027000001000000000000000300000001000000010000000100000072550000a5c4316200000000000000000000000000000000ff7f0000

In this case, the end lsn pointer specified by the user is
'0/01000029'. There is only one WAL record which starts before this
specified end lsn pointer whose start pointer is at 01000028, but that
WAL record ends at 0/100009F which is way beyond the specified end
lsn. So, how come we are able to display the complete WAL record info?
AFAIU, end lsn is the lsn pointer where you need to stop reading the
WAL data. If that is true, then there exists no valid WAL record
between the start and end lsn in this particular case.

Thanks Ashutosh, it's an edge case and I don't think we would want to
show a WAL record that ends at LSN after the user specified end-lsn
which doesn't look good. I fixed it in the v11 patch set. Now, the
pg_get_wal_records_info will show records only upto user specified
end_lsn, it doesn't show the last record which starts at LSN < end_lsn
but ends at LSN > end_lsn, see [1]postgres=# select start_lsn, end_lsn, prev_lsn from pg_get_wal_records_info('0/01000028', '0/01000029'); start_lsn | end_lsn | prev_lsn -----------+---------+---------- (0 rows).

Please review the v11 patch set further.

[1]: postgres=# select start_lsn, end_lsn, prev_lsn from pg_get_wal_records_info('0/01000028', '0/01000029'); start_lsn | end_lsn | prev_lsn -----------+---------+---------- (0 rows)
postgres=# select start_lsn, end_lsn, prev_lsn from
pg_get_wal_records_info('0/01000028', '0/01000029');
start_lsn | end_lsn | prev_lsn
-----------+---------+----------
(0 rows)

postgres=# select start_lsn, end_lsn, prev_lsn from
pg_get_wal_records_info('0/01000028', '0/100009F');
start_lsn | end_lsn | prev_lsn
-----------+-----------+----------
0/1000028 | 0/100009F | 0/0
(1 row)

postgres=# select start_lsn, end_lsn, prev_lsn from
pg_get_wal_records_info('0/01000028', '0/10000A0');
start_lsn | end_lsn | prev_lsn
-----------+-----------+----------
0/1000028 | 0/100009F | 0/0
(1 row)

postgres=# select start_lsn, end_lsn, prev_lsn from
pg_get_wal_records_info('0/01000028', '0/0100009E');
start_lsn | end_lsn | prev_lsn
-----------+---------+----------
(0 rows)

Regards,
Bharath Rupireddy.

Attachments:

v11-0001-pg_walinspect.patchapplication/octet-stream; name=v11-0001-pg_walinspect.patchDownload
From ad096719f321364a32b171fda4a21c26f04db3a1 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 17 Mar 2022 07:47:16 +0000
Subject: [PATCH v11] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql |  95 +++
 contrib/pg_walinspect/pg_walinspect.c        | 594 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  14 +-
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/common/relpath.c                         |  18 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/common/relpath.h                 |   1 +
 13 files changed, 758 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 332b486ecc..bbf220407b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -41,6 +41,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..14d7642350
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,95 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT record_length int4,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_length int8,
+    OUT record_length_percentage float4,
+    OUT fpi_length int8,
+    OUT fpi_length_percentage float4
+    )
+RETURNS SETOF record AS $$
+SELECT resource_manager,
+    count(*) AS cnt,
+    CASE WHEN count(*) > 0 THEN (count(*) * 100 / sum(count(*)) OVER total)::numeric(5,2) ELSE 0 END AS "count_%",
+    sum(record_length) AS trecl,
+    CASE WHEN sum(record_length) > 0 THEN (sum(record_length) * 100 / sum(sum(record_length)) OVER total)::numeric(5,2) ELSE 0 END AS "trecl_%",
+    sum(fpi_length) AS tfpil,
+    CASE WHEN sum(fpi_length) > 0 THEN (sum(fpi_length) * 100 / sum(sum(fpi_length)) OVER total)::numeric(5,2) ELSE 0 END AS "tfpil_%"
+FROM pg_get_wal_records_info(start_lsn, end_lsn)
+GROUP BY resource_manager
+WINDOW total AS ();
+$$ LANGUAGE SQL CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..667d9567cc
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,594 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+
+static XLogRecPtr ValidateInputLSN(XLogRecPtr lsn);
+static XLogRecPtr ValidateStartAndEndLSNs(XLogRecPtr start_lsn,
+										  XLogRecPtr end_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void GetWALRecordsInfoInternal(FunctionCallInfo fcinfo,
+									  XLogRecPtr start_lsn,
+									  XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Validate given LSN and return the LSN up to which the server has WAL.
+ */
+static XLogRecPtr
+ValidateInputLSN(XLogRecPtr lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate input WAL LSN. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL LSN")));
+
+	/*
+	 * We determine the current LSN of the server similar to how page_read
+	 * callback read_local_xlog_page does.
+	 */
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(curr_lsn));
+
+	if (lsn >= curr_lsn)
+	{
+		/*
+	 	 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+	 	 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	return curr_lsn;
+}
+
+/*
+ * Validate given start LSN and end LSN, return the new end LSN in case user
+ * hasn't specified one.
+ */
+static XLogRecPtr
+ValidateStartAndEndLSNs(XLogRecPtr start_lsn, XLogRecPtr end_lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate WAL start LSN. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL start LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	if (start_lsn >= curr_lsn)
+	{
+		/*
+	 	 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+	 	 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/*
+	 * If end_lsn is specified, let's ensure that it's not a future LSN i.e.
+	 * something the database system doesn't know about.
+	 */
+	if (!XLogRecPtrIsInvalid(end_lsn) &&
+		(end_lsn >= curr_lsn))
+	{
+		/*
+	 	 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+	 	 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/*
+	 * When end_lsn is not specified let's read up to the last WAL position
+	 * known to be on the server.
+	 */
+	if (XLogRecPtrIsInvalid(end_lsn))
+		end_lsn = curr_lsn;
+
+	Assert(!XLogRecPtrIsInvalid(end_lsn));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	return end_lsn;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get raw WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 5
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	*raw_record;
+	uint32	rec_len;
+	char	*raw_record_data;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(xlogreader->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(xlogreader));
+	values[i++] = UInt32GetDatum(rec_len);
+	values[i++] = PointerGetDatum(raw_record);
+
+	XLogReaderFree(xlogreader);
+
+	Assert(i == PG_GET_RAW_WAL_RECORD_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_RAW_WAL_RECORD_COLS
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int	block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	initStringInfo(&rec_desc);
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+	pfree(temp.data);
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		else
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			/* Calculate the amount of FPI data in the record. */
+			fpi_len += record->blocks[block_id].bimg_len;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}
+
+	main_data_len = XLogRecGetDataLen(record);
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = PointerGetDatum(data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+					  PG_GET_WAL_RECORD_INFO_COLS);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+		{
+			GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+						 	  PG_GET_WAL_RECORDS_INFO_COLS);
+
+			tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+		}
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future end WAL LSN i.e. WAL LSN the
+ * database system doesn't know about is specified.
+ *
+ * This function will figure out the end LSN if it's not specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	end_lsn = ValidateStartAndEndLSNs(start_lsn, end_lsn);
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index b7c06da255..1b072f9c11 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -959,13 +959,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1083,6 +1076,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index f128050b4e..aac41a0793 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 09f6464331..3e644372f9 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 0e94833129..8468732d76 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 477f0efe26..d62d6ce7f8 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -276,9 +276,7 @@ extern void XLogReaderFree(XLogReaderState *state);
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Read the next XLog record. Returns NULL on end-of-WAL or failure */
 extern struct XLogRecord *XLogReadRecord(XLogReaderState *state,
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v11-0001-pg_walinspect-tests.patchapplication/octet-stream; name=v11-0001-pg_walinspect-tests.patchDownload
From 7ca64e56418fdedbca4bb01b128a39bdf5349425 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 17 Mar 2022 07:51:14 +0000
Subject: [PATCH v11] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 171 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 132 ++++++++++++++
 2 files changed, 303 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..9fdce5ae6d
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,171 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 5);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 5);
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+CONTEXT:  SQL function "pg_get_wal_stats" statement 1
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+CONTEXT:  SQL function "pg_get_wal_stats" statement 1
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+-- Function accessible by users with role pg_read_server_files
+GRANT pg_read_server_files TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+-- Functions accessible by users with role pg_monitor
+GRANT pg_monitor TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..beea18e9fa
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,132 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 5);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 5);
+
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+-- Function accessible by users with role pg_read_server_files
+
+GRANT pg_read_server_files TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+
+-- Functions accessible by users with role pg_monitor
+
+GRANT pg_monitor TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v11-0001-pg_walinspect-docs.patchapplication/octet-stream; name=v11-0001-pg_walinspect-docs.patchDownload
From d9e368de1fc01e8eedf60b18eee5a064c4817571 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 17 Mar 2022 02:55:10 +0000
Subject: [PATCH v11] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 202 +++++++++++++++++++++++++++++++++
 3 files changed, 204 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index 1e42ce1a7f..4e7b87a42f 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -131,6 +131,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index fd853af01f..34c19c80f1 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -147,6 +147,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..d47d98b91e
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,202 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_raw_wal_record(in_lsn pg_lsn,
+                            start_lsn OUT pg_lsn,
+                            end_lsn OUT pg_lsn,
+                            record_length OUT int4,
+                            record OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. This function emits an error if
+      a future (the LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_read_server_files</literal> role. Access may be
+       granted by superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text,
+                             data_length OUT int4,
+                             data OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of the given LSN. This function emits an
+      error if a future (the LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text,
+                              data_length OUT int4,
+                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between <replaceable>start_lsn</replaceable>
+      and <replaceable>end_lsn</replaceable>. Returns one row per each valid
+      WAL record. This function figures out the <replaceable>end_lsn</replaceable>
+      if it's not specified, that means, it returns information up to the end
+      of WAL. Default value of <replaceable>end_lsn</replaceable> is
+      <literal>NULL</literal>. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>start_lsn</replaceable>
+      or <replaceable>end_lsn</replaceable> is specified. For example:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_length, fpi_length, block_ref, data_length from pg_get_wal_records_info('0/1602790', '0/16038D0');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager | record_length | fpi_length |                                block_ref                                 | data_length 
+-----------+-----------+-----------+-----+------------------+---------------+------------+--------------------------------------------------------------------------+-------------
+ 0/1602790 | 0/16027C7 | 0/1602758 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/16027C8 | 0/160283F | 0/1602790 |   0 | XLOG             |           114 |          0 |                                                                          |          88
+ 0/1602840 | 0/1602877 | 0/16027C8 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/1602878 | 0/16028AF | 0/1602840 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/16028B0 | 0/1602927 | 0/1602878 |   0 | XLOG             |           114 |          0 |                                                                          |          88
+ 0/1602928 | 0/16037AF | 0/16028B0 | 751 | Heap             |          3714 |       3660 | blkref #0: rel 1663/5/24628 blk 0 (FPW); hole: offset: 428, length: 4532 |           3
+ 0/16037B0 | 0/16037EF | 0/1602928 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/16037F0 | 0/160382F | 0/16037B0 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/1603830 | 0/160386F | 0/16037F0 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/1603870 | 0/16038A7 | 0/1603830 |   0 | Standby          |            54 |          0 |                                                                          |          28
+ 0/16038A8 | 0/16038CF | 0/1603870 | 751 | Transaction      |            34 |          0 |                                                                          |           8
+(11 rows)
+</screen>
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       resource_manager OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between <replaceable>start_lsn</replaceable>
+      and <replaceable>end_lsn</replaceable>. Returns one row per each <replaceable>resource_manager</replaceable>
+      type. This function figures out the <replaceable>end_lsn</replaceable> if
+      it's not specified, that means, it returns information up to the end of
+      WAL. Default value of <replaceable>end_lsn</replaceable> is
+      <literal>NULL</literal>. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>start_lsn</replaceable>
+      or <replaceable>end_lsn</replaceable> is specified. For example:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/15E80C0', '0/1600000');
+ resource_manager | count | count_percentage | record_length | record_length_percentage | fpi_length | fpi_length_percentage 
+------------------+-------+------------------+---------------+--------------------------+------------+-----------------------
+ Storage          |     1 |              0.7 |            42 |                     0.04 |          0 |                     0
+ Btree            |    27 |            18.88 |         65116 |                    66.98 |      63456 |                 72.45
+ Heap             |   101 |            70.63 |          7697 |                     7.92 |       1596 |                  1.82
+ Transaction      |     1 |              0.7 |           405 |                     0.42 |          0 |                     0
+ Heap2            |     6 |              4.2 |         23573 |                    24.25 |      22532 |                 25.73
+ Standby          |     5 |              3.5 |           242 |                     0.25 |          0 |                     0
+ XLOG             |     2 |              1.4 |           144 |                     0.15 |          0 |                     0
+(7 rows)
+</screen>
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#66Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Kyotaro Horiguchi (#64)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Mar 17, 2022 at 10:48 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

It still suggests unspecifiable end-LSN..

select * from pg_get_wal_records_info('4/4B28EB68', '4/4C000060');
ERROR: cannot accept future end LSN
DETAIL: Last known WAL LSN on the database system is 4/4C000060.

Thanks Kyotaro-san. We can change the detail message to show (current
flush lsn/last replayed lsn - 1), that's what I've done in v11 posted
upthread at [1]/messages/by-id/CALj2ACU8XjbYbMwh5x6hEUJdpRoG9=PO52_tuOSf1=MO7WtsmQ@mail.gmail.com. The problem is that all the pg_walinspect functions
would wait for the first valid record in read_local_xlog_page() via
InitXLogReaderState()->XLogFindNextRecord(), see[2]postgres=# select pg_current_wal_flush_lsn(); pg_current_wal_flush_lsn -------------------------- 0/1624430 (1 row).

We have two things to do:
1) Just document the behaviour "pg_walinspect functions will wait for
the first valid WAL record if there is none found after the specified
input LSN/start LSN.". This seems easier but some may see it as a
problem.
2) Have read_local_xlog_page_2 which doesn't wait for future WAL LSN
unlike read_local_xlog_page and like pg_waldump's WALDumpReadPage. It
requires a new function read_local_xlog_page_2 that almost looks like
read_local_xlog_page except wait (pg_usleep) loop, we can avoid code
duplication by moving the read_local_xlog_page code to a static
function read_local_xlog_page_guts(existing params, bool wait):

read_local_xlog_page(params)
read_local_xlog_page_guts(existing params, false);

read_local_xlog_page_2(params)
read_local_xlog_page_guts(existing params, true);

read_local_xlog_page_guts:
if (wait) wait for future wal; ---> existing pg_usleep code in
read_local_xlog_page.
else return;

I'm fine either way, please let me know your thoughts on this?

[1]: /messages/by-id/CALj2ACU8XjbYbMwh5x6hEUJdpRoG9=PO52_tuOSf1=MO7WtsmQ@mail.gmail.com
[2]: postgres=# select pg_current_wal_flush_lsn(); pg_current_wal_flush_lsn -------------------------- 0/1624430 (1 row)
postgres=# select pg_current_wal_flush_lsn();
pg_current_wal_flush_lsn
--------------------------
0/1624430
(1 row)

postgres=# select * from pg_get_wal_record_info('0/1624430');
ERROR: cannot accept future input LSN
DETAIL: Last known WAL LSN on the database system is 0/162442F.
postgres=# select * from pg_get_wal_record_info('0/162442f'); --->
waits for the first valid record in read_local_xlog_page.

Regards,
Bharath Rupireddy.

#67Nitin Jadhav
nitinjadhavpostgres@gmail.com
In reply to: Bharath Rupireddy (#66)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Hi Bharath,

Due to recent commits on master, the pg_walinpect module is not
compiling. Kindly update the patch.

pg_walinspect.c: In function ‘GetXLogRecordInfo’:
pg_walinspect.c:362:39: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘max_block_id’
362 | for (block_id = 0; block_id <= record->max_block_id; block_id++)
| ^~
pg_walinspect.c:382:29: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘blocks’
382 | uint8 bimg_info = record->blocks[block_id].bimg_info;
| ^~
pg_walinspect.c:385:21: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘blocks’
385 | fpi_len += record->blocks[block_id].bimg_len;
| ^~
pg_walinspect.c:402:16: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘blocks’
402 | record->blocks[block_id].hole_offset,
| ^~
pg_walinspect.c:403:16: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘blocks’
403 | record->blocks[block_id].hole_length,
| ^~
pg_walinspect.c:405:16: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘blocks’
405 | record->blocks[block_id].hole_length -
| ^~
pg_walinspect.c:406:16: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘blocks’
406 | record->blocks[block_id].bimg_len,
| ^~
pg_walinspect.c:414:16: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘blocks’
414 | record->blocks[block_id].hole_offset,
| ^~
pg_walinspect.c:415:16: error: ‘XLogReaderState’ {aka ‘struct
XLogReaderState’} has no member named ‘blocks’
415 | record->blocks[block_id].hole_length);
| ^~
make: *** [../../src/Makefile.global:941: pg_walinspect.o] Error 1

Thanks & Regards,
Nitin Jadhav

On Thu, Mar 17, 2022 at 1:54 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Show quoted text

On Thu, Mar 17, 2022 at 10:48 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

It still suggests unspecifiable end-LSN..

select * from pg_get_wal_records_info('4/4B28EB68', '4/4C000060');
ERROR: cannot accept future end LSN
DETAIL: Last known WAL LSN on the database system is 4/4C000060.

Thanks Kyotaro-san. We can change the detail message to show (current
flush lsn/last replayed lsn - 1), that's what I've done in v11 posted
upthread at [1]. The problem is that all the pg_walinspect functions
would wait for the first valid record in read_local_xlog_page() via
InitXLogReaderState()->XLogFindNextRecord(), see[2].

We have two things to do:
1) Just document the behaviour "pg_walinspect functions will wait for
the first valid WAL record if there is none found after the specified
input LSN/start LSN.". This seems easier but some may see it as a
problem.
2) Have read_local_xlog_page_2 which doesn't wait for future WAL LSN
unlike read_local_xlog_page and like pg_waldump's WALDumpReadPage. It
requires a new function read_local_xlog_page_2 that almost looks like
read_local_xlog_page except wait (pg_usleep) loop, we can avoid code
duplication by moving the read_local_xlog_page code to a static
function read_local_xlog_page_guts(existing params, bool wait):

read_local_xlog_page(params)
read_local_xlog_page_guts(existing params, false);

read_local_xlog_page_2(params)
read_local_xlog_page_guts(existing params, true);

read_local_xlog_page_guts:
if (wait) wait for future wal; ---> existing pg_usleep code in
read_local_xlog_page.
else return;

I'm fine either way, please let me know your thoughts on this?

[1] /messages/by-id/CALj2ACU8XjbYbMwh5x6hEUJdpRoG9=PO52_tuOSf1=MO7WtsmQ@mail.gmail.com
[2]
postgres=# select pg_current_wal_flush_lsn();
pg_current_wal_flush_lsn
--------------------------
0/1624430
(1 row)

postgres=# select * from pg_get_wal_record_info('0/1624430');
ERROR: cannot accept future input LSN
DETAIL: Last known WAL LSN on the database system is 0/162442F.
postgres=# select * from pg_get_wal_record_info('0/162442f'); --->
waits for the first valid record in read_local_xlog_page.

Regards,
Bharath Rupireddy.

#68Andres Freund
andres@anarazel.de
In reply to: Bharath Rupireddy (#65)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Hi,

First look at this patch, so I might be repeating stuff already commented on /
discussed.

On 2022-03-17 13:25:35 +0530, Bharath Rupireddy wrote:

+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT record_length int4,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;

What is raw about the function?

Why "CALLED ON NULL INPUT"? It doesn't make sense to call the function with a
NULL lsn, does it? Also, that's the default, why is it restated here?

+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;

I don't think it's appropriate for pg_monitor to see all the data in the WAL.

+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_length int8,
+    OUT record_length_percentage float4,
+    OUT fpi_length int8,
+    OUT fpi_length_percentage float4
+    )
+RETURNS SETOF record AS $$
+SELECT resource_manager,
+    count(*) AS cnt,
+    CASE WHEN count(*) > 0 THEN (count(*) * 100 / sum(count(*)) OVER total)::numeric(5,2) ELSE 0 END AS "count_%",
+    sum(record_length) AS trecl,
+    CASE WHEN sum(record_length) > 0 THEN (sum(record_length) * 100 / sum(sum(record_length)) OVER total)::numeric(5,2) ELSE 0 END AS "trecl_%",
+    sum(fpi_length) AS tfpil,
+    CASE WHEN sum(fpi_length) > 0 THEN (sum(fpi_length) * 100 / sum(sum(fpi_length)) OVER total)::numeric(5,2) ELSE 0 END AS "tfpil_%"
+FROM pg_get_wal_records_info(start_lsn, end_lsn)
+GROUP BY resource_manager
+WINDOW total AS ();
+$$ LANGUAGE SQL CALLED ON NULL INPUT PARALLEL SAFE;

This seems like an exceedingly expensive way to compute this. Not just because
of doing the grouping, window etc, but also because it's serializing the
"data" field from pg_get_wal_records_info() just to never use it. With any
appreciatable amount of data the return value pg_get_wal_records_info() will
be serialized into a on-disk tuplestore.

This is probably close to an order of magnitude slower than pg_waldump
--stats. Which imo renders this largely useless.

The column names don't seem great either. "tfpil"?

+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}

Why have this stuff if it's not used?

+/*
+ * Validate given LSN and return the LSN up to which the server has WAL.
+ */
+static XLogRecPtr
+ValidateInputLSN(XLogRecPtr lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate input WAL LSN. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL LSN")));
+
+	/*
+	 * We determine the current LSN of the server similar to how page_read
+	 * callback read_local_xlog_page does.
+	 */
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(curr_lsn));
+
+	if (lsn >= curr_lsn)
+	{
+		/*
+	 	 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+	 	 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+	return curr_lsn;
+}
+
+/*
+ * Validate given start LSN and end LSN, return the new end LSN in case user
+ * hasn't specified one.
+ */
+static XLogRecPtr
+ValidateStartAndEndLSNs(XLogRecPtr start_lsn, XLogRecPtr end_lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate WAL start LSN. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL start LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	if (start_lsn >= curr_lsn)
+	{
+		/*
+	 	 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+	 	 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+	/*
+	 * If end_lsn is specified, let's ensure that it's not a future LSN i.e.
+	 * something the database system doesn't know about.
+	 */
+	if (!XLogRecPtrIsInvalid(end_lsn) &&
+		(end_lsn >= curr_lsn))
+	{
+		/*
+	 	 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+	 	 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/*
+	 * When end_lsn is not specified let's read up to the last WAL position
+	 * known to be on the server.
+	 */
+	if (XLogRecPtrIsInvalid(end_lsn))
+		end_lsn = curr_lsn;
+
+	Assert(!XLogRecPtrIsInvalid(end_lsn));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	return end_lsn;
+}

These two functions are largely redundant, that doesn't seem great.

+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 5
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	*raw_record;
+	uint32	rec_len;
+	char	*raw_record_data;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+

Most of this has another copy in pg_get_wal_record_info(). Can more of this be
deduplicated?

+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int	block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	initStringInfo(&rec_desc);
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+	pfree(temp.data);
+	initStringInfo(&rec_blk_ref);

This seems unnecessarily wasteful. You serialize into one stringinfo, just to
then copy that stringinfo into another stringinfo. Just to then allocate yet
another stringinfo.

+ /* Block references (detailed format). */

This comment seems copied from pg_waldump, but doesn't make sense here,
because there's no short format.

+	for (block_id = 0; block_id <= record->max_block_id; block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		else
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = record->blocks[block_id].bimg_info;
+
+			/* Calculate the amount of FPI data in the record. */
+			fpi_len += record->blocks[block_id].bimg_len;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length,
+								 BLCKSZ -
+								 record->blocks[block_id].hole_length -
+								 record->blocks[block_id].bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 record->blocks[block_id].hole_offset,
+								 record->blocks[block_id].hole_length);
+			}
+		}
+	}

To me duplicating this much code from waldump seems like a bad idea from a
maintainability POV.

+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;

This should likely use the infrastructure introduced in 5b81703787bfc1e6072c8e37125eba0c5598b807.

+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+		{
+			GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+						 	  PG_GET_WAL_RECORDS_INFO_COLS);
+
+			tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+		}
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;

Seems weird to have both of these conditions separately.

Greetings,

Andres Freund

#69Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Nitin Jadhav (#67)
3 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Mar 18, 2022 at 8:07 PM Nitin Jadhav
<nitinjadhavpostgres@gmail.com> wrote:

Hi Bharath,

Due to recent commits on master, the pg_walinpect module is not
compiling. Kindly update the patch.

Thanks Nitin. Here's an updated v12 patch-set. I will respond to
Andres comments in a while.

Regards,
Bharath Rupireddy.

Attachments:

v12-0001-pg_walinspect.patchapplication/octet-stream; name=v12-0001-pg_walinspect.patchDownload
From 7243e163670037069e6d4d874d734a751130d3bb Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sun, 20 Mar 2022 08:49:03 +0000
Subject: [PATCH v12] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql |  95 +++
 contrib/pg_walinspect/pg_walinspect.c        | 594 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  14 +-
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/common/relpath.c                         |  18 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/common/relpath.h                 |   1 +
 13 files changed, 758 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 332b486ecc..bbf220407b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -41,6 +41,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..14d7642350
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,95 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_raw_wal_record()
+--
+CREATE FUNCTION pg_get_raw_wal_record(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT record_length int4,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_raw_wal_record'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_monitor;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn DEFAULT NULL,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_length int8,
+    OUT record_length_percentage float4,
+    OUT fpi_length int8,
+    OUT fpi_length_percentage float4
+    )
+RETURNS SETOF record AS $$
+SELECT resource_manager,
+    count(*) AS cnt,
+    CASE WHEN count(*) > 0 THEN (count(*) * 100 / sum(count(*)) OVER total)::numeric(5,2) ELSE 0 END AS "count_%",
+    sum(record_length) AS trecl,
+    CASE WHEN sum(record_length) > 0 THEN (sum(record_length) * 100 / sum(sum(record_length)) OVER total)::numeric(5,2) ELSE 0 END AS "trecl_%",
+    sum(fpi_length) AS tfpil,
+    CASE WHEN sum(fpi_length) > 0 THEN (sum(fpi_length) * 100 / sum(sum(fpi_length)) OVER total)::numeric(5,2) ELSE 0 END AS "tfpil_%"
+FROM pg_get_wal_records_info(start_lsn, end_lsn)
+GROUP BY resource_manager
+WINDOW total AS ();
+$$ LANGUAGE SQL CALLED ON NULL INPUT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_monitor;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..2e9d48b785
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,594 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+extern void _PG_init(void);
+extern void _PG_fini(void);
+
+PG_FUNCTION_INFO_V1(pg_get_raw_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+
+static XLogRecPtr ValidateInputLSN(XLogRecPtr lsn);
+static XLogRecPtr ValidateStartAndEndLSNs(XLogRecPtr start_lsn,
+										  XLogRecPtr end_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void GetWALRecordsInfoInternal(FunctionCallInfo fcinfo,
+									  XLogRecPtr start_lsn,
+									  XLogRecPtr end_lsn);
+
+/*
+ * Module load callback.
+ */
+void
+_PG_init(void)
+{
+	/* Define custom GUCs and install hooks here, if any. */
+
+	/*
+	 * Have EmitWarningsOnPlaceholders("pg_walinspect"); if custom GUCs are
+	 * defined.
+	 */
+}
+
+/*
+ * Module unload callback.
+ */
+void
+_PG_fini(void)
+{
+	/* Uninstall hooks, if any. */
+}
+
+/*
+ * Validate given LSN and return the LSN up to which the server has WAL.
+ */
+static XLogRecPtr
+ValidateInputLSN(XLogRecPtr lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate input WAL LSN. */
+	if (XLogRecPtrIsInvalid(lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL LSN")));
+
+	/*
+	 * We determine the current LSN of the server similar to how page_read
+	 * callback read_local_xlog_page does.
+	 */
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(curr_lsn));
+
+	if (lsn >= curr_lsn)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	return curr_lsn;
+}
+
+/*
+ * Validate given start LSN and end LSN, return the new end LSN in case user
+ * hasn't specified one.
+ */
+static XLogRecPtr
+ValidateStartAndEndLSNs(XLogRecPtr start_lsn, XLogRecPtr end_lsn)
+{
+	XLogRecPtr curr_lsn;
+
+	/* Validate WAL start LSN. */
+	if (XLogRecPtrIsInvalid(start_lsn))
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("invalid WAL start LSN")));
+
+	if (!RecoveryInProgress())
+		curr_lsn = GetFlushRecPtr(NULL);
+	else
+		curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	if (start_lsn >= curr_lsn)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/*
+	 * If end_lsn is specified, let's ensure that it's not a future LSN i.e.
+	 * something the database system doesn't know about.
+	 */
+	if (!XLogRecPtrIsInvalid(end_lsn) &&
+		(end_lsn >= curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last known WAL LSN on the database system is %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/*
+	 * When end_lsn is not specified let's read up to the last WAL position
+	 * known to be on the server.
+	 */
+	if (XLogRecPtrIsInvalid(end_lsn))
+		end_lsn = curr_lsn;
+
+	Assert(!XLogRecPtrIsInvalid(end_lsn));
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	return end_lsn;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get raw WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_RAW_WAL_RECORD_COLS 5
+	XLogRecPtr	lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	bytea	*raw_record;
+	uint32	rec_len;
+	char	*raw_record_data;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_RAW_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_RAW_WAL_RECORD_COLS];
+	int	i = 0;
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	rec_len = XLogRecGetTotalLen(xlogreader);
+
+	Assert(rec_len > 0);
+
+	raw_record = (bytea *) palloc(rec_len + VARHDRSZ);
+	SET_VARSIZE(raw_record, rec_len + VARHDRSZ);
+	raw_record_data = VARDATA(raw_record);
+
+	memcpy(raw_record_data, record, rec_len);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	values[i++] = LSNGetDatum(first_record);
+	values[i++] = LSNGetDatum(xlogreader->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(xlogreader));
+	values[i++] = UInt32GetDatum(rec_len);
+	values[i++] = PointerGetDatum(raw_record);
+
+	XLogReaderFree(xlogreader);
+
+	Assert(i == PG_GET_RAW_WAL_RECORD_COLS);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_RAW_WAL_RECORD_COLS
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int	block_id;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	StringInfoData temp;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	initStringInfo(&rec_desc);
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	initStringInfo(&temp);
+	desc->rm_desc(&temp, record);
+	appendStringInfo(&rec_desc, "%s", temp.data);
+	pfree(temp.data);
+	initStringInfo(&rec_blk_ref);
+
+	/* Block references (detailed format). */
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (forknum != MAIN_FORKNUM)
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u fork %s blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, get_forkname(forknum), blk);
+		else
+			appendStringInfo(&rec_blk_ref,
+							"blkref #%u: rel %u/%u/%u blk %u",
+							block_id, rnode.spcNode, rnode.dbNode,
+							rnode.relNode, blk);
+
+		if (XLogRecHasBlockImage(record, block_id))
+		{
+			uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
+
+			/* Calculate the amount of FPI data in the record. */
+			fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+			if (BKPIMAGE_COMPRESSED(bimg_info))
+			{
+				const char *method;
+
+				if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+					method = "pglz";
+				else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+					method = "lz4";
+				else
+					method = "unknown";
+
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u, "
+								 "compression saved: %u, method: %s",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 XLogRecGetBlock(record, block_id)->hole_offset,
+								 XLogRecGetBlock(record, block_id)->hole_length,
+								 BLCKSZ -
+								 XLogRecGetBlock(record, block_id)->hole_length -
+								 XLogRecGetBlock(record, block_id)->bimg_len,
+								 method);
+			}
+			else
+			{
+				appendStringInfo(&rec_blk_ref, " (FPW%s); hole: offset: %u, length: %u",
+								 XLogRecBlockImageApply(record, block_id) ?
+								 "" : " for WAL verification",
+								 XLogRecGetBlock(record, block_id)->hole_offset,
+								 XLogRecGetBlock(record, block_id)->hole_length);
+			}
+		}
+	}
+
+	main_data_len = XLogRecGetDataLen(record);
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = PointerGetDatum(data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	XLogRecPtr	lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	Datum	result;
+	HeapTuple	tuple;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	lsn = PG_GETARG_LSN(0);
+
+	(void) ValidateInputLSN(lsn);
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+					  PG_GET_WAL_RECORD_INFO_COLS);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+
+	/* Check to see if caller supports us returning a tuplestore. */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	/* Build tuplestore to hold the result rows. */
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+		{
+			GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+							  PG_GET_WAL_RECORDS_INFO_COLS);
+
+			tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+		}
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	/* Clean up and return the tuplestore. */
+	tuplestore_donestoring(tupstore);
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future end WAL LSN i.e. WAL LSN the
+ * database system doesn't know about is specified.
+ *
+ * This function will figure out the end LSN if it's not specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+	end_lsn = PG_GETARG_LSN(1);
+
+	end_lsn = ValidateStartAndEndLSNs(start_lsn, end_lsn);
+
+	GetWALRecordsInfoInternal(fcinfo, start_lsn, end_lsn);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index e437c42992..9a79173ccc 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -1320,13 +1320,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1447,6 +1440,13 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
+
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index fc081adfb8..53b40d2bd1 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..e8e3c44eae 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -40,6 +40,24 @@ const char *const forkNames[] = {
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
 
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
+
 /*
  * forkname_to_number - look up fork number by name
  *
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 09f6464331..3e644372f9 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 0e94833129..8468732d76 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index f4388cc9be..b4c7d93787 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -340,9 +340,7 @@ extern void XLogReaderSetDecodeBuffer(XLogReaderState *state,
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Return values from XLogPageReadCB. */
 typedef enum XLogPageReadResult
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..3743f2e505 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -57,6 +57,7 @@ typedef enum ForkNumber
 #define FORKNAMECHARS	4		/* max chars for a fork name */
 
 extern const char *const forkNames[];
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
-- 
2.25.1

v12-0001-pg_walinspect-tests.patchapplication/octet-stream; name=v12-0001-pg_walinspect-tests.patchDownload
From 7d8b745376956b0489e4b4a716ae0561a4c8fdd8 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sun, 20 Mar 2022 08:51:13 +0000
Subject: [PATCH v12] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 171 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 132 ++++++++++++++
 2 files changed, 303 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..9fdce5ae6d
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,171 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 5);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 5);
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+ERROR:  invalid WAL LSN
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+ERROR:  invalid WAL start LSN
+CONTEXT:  SQL function "pg_get_wal_stats" statement 1
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+CONTEXT:  SQL function "pg_get_wal_stats" statement 1
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+-- Function accessible by users with role pg_read_server_files
+GRANT pg_read_server_files TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+-- Functions accessible by users with role pg_monitor
+GRANT pg_monitor TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..beea18e9fa
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,132 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 5);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 5);
+
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+
+SELECT pg_get_raw_wal_record('0/0'); -- ERROR
+
+SELECT pg_get_wal_record_info('0/0'); -- ERROR
+
+SELECT pg_get_wal_records_info('0/0', '0/0'); -- ERROR
+
+SELECT pg_get_wal_stats('0/0', '0/0'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_raw_wal_record(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+-- Function accessible by users with role pg_read_server_files
+
+GRANT pg_read_server_files TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+
+-- Functions accessible by users with role pg_monitor
+
+GRANT pg_monitor TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE pg_monitor FROM regress_pg_walinspect;
+
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_raw_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_raw_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v12-0001-pg_walinspect-docs.patchapplication/octet-stream; name=v12-0001-pg_walinspect-docs.patchDownload
From b9bd131f12e4bec77ab35525579e14cbc418286a Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sun, 20 Mar 2022 08:51:50 +0000
Subject: [PATCH v12] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 202 +++++++++++++++++++++++++++++++++
 3 files changed, 204 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index 1e42ce1a7f..4e7b87a42f 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -131,6 +131,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index fd853af01f..34c19c80f1 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -147,6 +147,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..d47d98b91e
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,202 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_raw_wal_record(in_lsn pg_lsn,
+                            start_lsn OUT pg_lsn,
+                            end_lsn OUT pg_lsn,
+                            record_length OUT int4,
+                            record OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. This function emits an error if
+      a future (the LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_read_server_files</literal> role. Access may be
+       granted by superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text,
+                             data_length OUT int4,
+                             data OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of the given LSN. This function emits an
+      error if a future (the LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text,
+                              data_length OUT int4,
+                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between <replaceable>start_lsn</replaceable>
+      and <replaceable>end_lsn</replaceable>. Returns one row per each valid
+      WAL record. This function figures out the <replaceable>end_lsn</replaceable>
+      if it's not specified, that means, it returns information up to the end
+      of WAL. Default value of <replaceable>end_lsn</replaceable> is
+      <literal>NULL</literal>. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>start_lsn</replaceable>
+      or <replaceable>end_lsn</replaceable> is specified. For example:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_length, fpi_length, block_ref, data_length from pg_get_wal_records_info('0/1602790', '0/16038D0');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager | record_length | fpi_length |                                block_ref                                 | data_length 
+-----------+-----------+-----------+-----+------------------+---------------+------------+--------------------------------------------------------------------------+-------------
+ 0/1602790 | 0/16027C7 | 0/1602758 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/16027C8 | 0/160283F | 0/1602790 |   0 | XLOG             |           114 |          0 |                                                                          |          88
+ 0/1602840 | 0/1602877 | 0/16027C8 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/1602878 | 0/16028AF | 0/1602840 |   0 | Standby          |            50 |          0 |                                                                          |          24
+ 0/16028B0 | 0/1602927 | 0/1602878 |   0 | XLOG             |           114 |          0 |                                                                          |          88
+ 0/1602928 | 0/16037AF | 0/16028B0 | 751 | Heap             |          3714 |       3660 | blkref #0: rel 1663/5/24628 blk 0 (FPW); hole: offset: 428, length: 4532 |           3
+ 0/16037B0 | 0/16037EF | 0/1602928 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/16037F0 | 0/160382F | 0/16037B0 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/1603830 | 0/160386F | 0/16037F0 | 751 | Heap             |            59 |          0 | blkref #0: rel 1663/5/24628 blk 0                                        |           3
+ 0/1603870 | 0/16038A7 | 0/1603830 |   0 | Standby          |            54 |          0 |                                                                          |          28
+ 0/16038A8 | 0/16038CF | 0/1603870 | 751 | Transaction      |            34 |          0 |                                                                          |           8
+(11 rows)
+</screen>
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       resource_manager OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between <replaceable>start_lsn</replaceable>
+      and <replaceable>end_lsn</replaceable>. Returns one row per each <replaceable>resource_manager</replaceable>
+      type. This function figures out the <replaceable>end_lsn</replaceable> if
+      it's not specified, that means, it returns information up to the end of
+      WAL. Default value of <replaceable>end_lsn</replaceable> is
+      <literal>NULL</literal>. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>start_lsn</replaceable>
+      or <replaceable>end_lsn</replaceable> is specified. For example:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/15E80C0', '0/1600000');
+ resource_manager | count | count_percentage | record_length | record_length_percentage | fpi_length | fpi_length_percentage 
+------------------+-------+------------------+---------------+--------------------------+------------+-----------------------
+ Storage          |     1 |              0.7 |            42 |                     0.04 |          0 |                     0
+ Btree            |    27 |            18.88 |         65116 |                    66.98 |      63456 |                 72.45
+ Heap             |   101 |            70.63 |          7697 |                     7.92 |       1596 |                  1.82
+ Transaction      |     1 |              0.7 |           405 |                     0.42 |          0 |                     0
+ Heap2            |     6 |              4.2 |         23573 |                    24.25 |      22532 |                 25.73
+ Standby          |     5 |              3.5 |           242 |                     0.25 |          0 |                     0
+ XLOG             |     2 |              1.4 |           144 |                     0.15 |          0 |                     0
+(7 rows)
+</screen>
+     </para>
+
+      <para>
+       By default, use of this function is restricted to superusers and members
+       of the <literal>pg_monitor</literal> role. Access may be granted by
+       superusers to others using <command>GRANT</command>.
+      </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#70Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Andres Freund (#68)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Sat, Mar 19, 2022 at 5:18 AM Andres Freund <andres@anarazel.de> wrote:

Hi,

First look at this patch, so I might be repeating stuff already commented on /
discussed.

Thanks for taking a look at the patch.

On 2022-03-17 13:25:35 +0530, Bharath Rupireddy wrote:

+--
+-- pg_get_raw_wal_record()

What is raw about the function?

It right now gives data starting from the output of XLogReadRecord
upto XLogRecGetTotalLen(xlogreader); length. Given that XLogReadRecord
returns a pointer to the decoded record's header, I'm not sure it's
the right choice. Actually, this function's intention(not an immediate
use-case though), is to feed the WAL record to another function and
then, say, repair a corrupted page given a base data page.

As I said upthread, I'm open to removing this function for now, when a
realistic need comes we can add it back. It also raised some concerns
around the security and permissions. Thoughts?

Why "CALLED ON NULL INPUT"? It doesn't make sense to call the function with a
NULL lsn, does it? Also, that's the default, why is it restated here?

pg_get_wal_records_info needed that option (if end_lsn being the
default, providing WAL info upto the end of WAL). Also, we can emit
better error message ("invalid WAL start LSN") instead of generic one.
I wanted to keep error message and code same across all the functions
hence CALLED ON NULL INPUT option for pg_get_raw_wal_record.

+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;

I don't think it's appropriate for pg_monitor to see all the data in the WAL.

How about pg_read_server_files or some other?

+-- pg_get_wal_stats()

This seems like an exceedingly expensive way to compute this. Not just because
of doing the grouping, window etc, but also because it's serializing the
"data" field from pg_get_wal_records_info() just to never use it. With any
appreciatable amount of data the return value pg_get_wal_records_info() will
be serialized into a on-disk tuplestore.

This is probably close to an order of magnitude slower than pg_waldump
--stats. Which imo renders this largely useless.

Yeah that's true. Do you suggest having pg_get_wal_stats() a
c-function like in v8 patch [1]/messages/by-id/CALj2ACWhcbW_s4BXLyCpLWcCppZN9ncTXHbn4dv1F0Vpe0kxqA@mail.gmail.com?

SEe some numbers at [2]with pg_get_wal_stats using pg_get_wal_stats: Time: 1394.919 ms (00:01.395) Time: 1403.199 ms (00:01.403) Time: 1408.138 ms (00:01.408) Time: 1397.670 ms (00:01.398) with pg_get_wal_stats using
pg_get_wal_records_info and pg_get_wal_records_info as a direct
c-function like in v8 patch [1]/messages/by-id/CALj2ACWhcbW_s4BXLyCpLWcCppZN9ncTXHbn4dv1F0Vpe0kxqA@mail.gmail.com. A direct c-function always fares
better (84 msec vs 1400msec).

The column names don't seem great either. "tfpil"?

"total fpi length" - tfpil wanted to keep it short - it's just an
internal column name isn't it? The actual column name the user sees is
fpi_length.

+void
+_PG_init(void)

+void
+_PG_fini(void)

Why have this stuff if it's not used?

I kept it as a placeholder for future code additions, for instance
test_decoding.c and ssl_passphrase_func.c has empty _PG_init(),
_PG_fini(). If okay, I can mention there like "placeholder for now",
otherwise I can remove it.

+static XLogRecPtr
+ValidateInputLSN(XLogRecPtr lsn)

+static XLogRecPtr
+ValidateStartAndEndLSNs(XLogRecPtr start_lsn, XLogRecPtr end_lsn)
+{

These two functions are largely redundant, that doesn't seem great.

I will modify it in the next version.

+Datum
+pg_get_raw_wal_record(PG_FUNCTION_ARGS)

Most of this has another copy in pg_get_wal_record_info(). Can more of this be
deduplicated?

I will do, if we decide on whether or not to have the
pg_get_raw_wal_record function at all? Please see my comments above.

+     initStringInfo(&temp);
+     desc->rm_desc(&temp, record);
+     appendStringInfo(&rec_desc, "%s", temp.data);
+     pfree(temp.data);
+     initStringInfo(&rec_blk_ref);

This seems unnecessarily wasteful. You serialize into one stringinfo, just to
then copy that stringinfo into another stringinfo. Just to then allocate yet
another stringinfo.

Yeah, I will remove it. Looks like all the rm_desc callbacks append to
the passed-in buffer and not reset it, so we should be good.

+ /* Block references (detailed format). */

This comment seems copied from pg_waldump, but doesn't make sense here,
because there's no short format.

Yes, I will remove it.

+     for (block_id = 0; block_id <= record->max_block_id; block_id++)
+     {

To me duplicating this much code from waldump seems like a bad idea from a
maintainability POV.

Even if we were to put the above code from pg_walinspect and
pg_waldump into, say, walutils.c or some other existing file, there we
had to make if (pg_walinspect) appendStringInfo else if (pg_waldump)
printf() sort of thing, isn't it clumsy? Also, unnecessary if
conditions need to be executed for every record. For maintainability,
I added a note in pg_walinspect.c and pg_waldump.c to consider fixing
things in both places (of course this might sound dumbest way of doing
it, IMHO, it's sensible, given the if(pg_walinspect)-else
if(pg_waldump) sorts of code that we need to do in the common
functions). Thoughts?

+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfoInternal(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,

This should likely use the infrastructure introduced in 5b81703787bfc1e6072c8e37125eba0c5598b807.

Yes, I will change it.

+     for (;;)
+     {
+             (void) ReadNextXLogRecord(xlogreader, first_record);
+
+             /*
+              * Let's not show the record info if it is spanning more than the
+              * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+              * use "end" here.
+              */
+             if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+             {
+                     GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+                                                       PG_GET_WAL_RECORDS_INFO_COLS);
+
+                     tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+             }
+
+             /* Exit loop if read up to end_lsn. */
+             if (xlogreader->EndRecPtr >= end_lsn)
+                     break;

Seems weird to have both of these conditions separately.

Yeah. It is to handle some edge cases to print the WAL upto end_lsn
and avoid waiting in read_local_xlog_page. I will change it.

Actually, there's an open point as specified in [3]/messages/by-id/CALj2ACVBST5Us6-eDz4q_Gem3rUHSC7AYNOB7tmp9Yqq6PHsXw@mail.gmail.com. Any thoughts on it?

[1]: /messages/by-id/CALj2ACWhcbW_s4BXLyCpLWcCppZN9ncTXHbn4dv1F0Vpe0kxqA@mail.gmail.com
[2]: with pg_get_wal_stats using pg_get_wal_stats: Time: 1394.919 ms (00:01.395) Time: 1403.199 ms (00:01.403) Time: 1408.138 ms (00:01.408) Time: 1397.670 ms (00:01.398)
Time: 1394.919 ms (00:01.395)
Time: 1403.199 ms (00:01.403)
Time: 1408.138 ms (00:01.408)
Time: 1397.670 ms (00:01.398)

with pg_get_wal_stats as a c-function like in v8 patch [1]/messages/by-id/CALj2ACWhcbW_s4BXLyCpLWcCppZN9ncTXHbn4dv1F0Vpe0kxqA@mail.gmail.com:
Time: 84.319 ms
Time: 84.303 ms
Time: 84.208 ms
Time: 84.452 ms

use case:
create extension pg_walinspect;

create table foo(col int);
insert into foo select * from generate_series(1, 100000);
update foo set col = col*2+1;
delete from foo;

\timing on
select * from pg_get_wal_stats('0/01000028');
\timing off

output:
postgres=# select * from pg_get_wal_stats('0/01000028');
resource_manager | count | count_percentage | record_length |
record_length_percentage | fpi_length | fpi_length_percentage
------------------+--------+------------------+---------------+--------------------------+------------+-----------------------
Storage | 13 | 0 | 546 |
0 | 0 | 0
CLOG | 1 | 0 | 30 |
0 | 0 | 0
Database | 2 | 0 | 84 |
0 | 0 | 0
Btree | 13078 | 3.1 | 1486990 |
4.97 | 461512 | 23.13
Heap | 404835 | 95.84 | 26354653 |
88.17 | 456576 | 22.88
Transaction | 721 | 0.17 | 178933 |
0.6 | 0 | 0
Heap2 | 3056 | 0.72 | 1131836 |
3.79 | 376932 | 18.89
Standby | 397 | 0.09 | 23226 |
0.08 | 0 | 0
XLOG | 316 | 0.07 | 716027 |
2.4 | 700164 | 35.09
(9 rows)

[3]: /messages/by-id/CALj2ACVBST5Us6-eDz4q_Gem3rUHSC7AYNOB7tmp9Yqq6PHsXw@mail.gmail.com

Regards,
Bharath Rupireddy.

#71Andres Freund
andres@anarazel.de
In reply to: Bharath Rupireddy (#70)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Hi,

On 2022-03-22 21:57:51 +0530, Bharath Rupireddy wrote:

On Sat, Mar 19, 2022 at 5:18 AM Andres Freund <andres@anarazel.de> wrote:

On 2022-03-17 13:25:35 +0530, Bharath Rupireddy wrote:

+--
+-- pg_get_raw_wal_record()

What is raw about the function?

It right now gives data starting from the output of XLogReadRecord
upto XLogRecGetTotalLen(xlogreader); length. Given that XLogReadRecord
returns a pointer to the decoded record's header, I'm not sure it's
the right choice. Actually, this function's intention(not an immediate
use-case though), is to feed the WAL record to another function and
then, say, repair a corrupted page given a base data page.

As I said upthread, I'm open to removing this function for now, when a
realistic need comes we can add it back. It also raised some concerns
around the security and permissions. Thoughts?

I'm ok with having it with appropriate permissions, I just don't like the
name.

Why "CALLED ON NULL INPUT"? It doesn't make sense to call the function with a
NULL lsn, does it? Also, that's the default, why is it restated here?

pg_get_wal_records_info needed that option (if end_lsn being the
default, providing WAL info upto the end of WAL). Also, we can emit
better error message ("invalid WAL start LSN") instead of generic one.
I wanted to keep error message and code same across all the functions
hence CALLED ON NULL INPUT option for pg_get_raw_wal_record.

I think it should be strict if it behaves strict. I fail to see what
consistency in error messages is worth here. And I'd probably just create two
different functions for begin and begin & end LSN and mark those as strict as
well.

+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_monitor;

I don't think it's appropriate for pg_monitor to see all the data in the WAL.

How about pg_read_server_files or some other?

That seems more appropriate.

+-- pg_get_wal_stats()

This seems like an exceedingly expensive way to compute this. Not just because
of doing the grouping, window etc, but also because it's serializing the
"data" field from pg_get_wal_records_info() just to never use it. With any
appreciatable amount of data the return value pg_get_wal_records_info() will
be serialized into a on-disk tuplestore.

This is probably close to an order of magnitude slower than pg_waldump
--stats. Which imo renders this largely useless.

Yeah that's true. Do you suggest having pg_get_wal_stats() a
c-function like in v8 patch [1]?

Yes.

SEe some numbers at [2] with pg_get_wal_stats using
pg_get_wal_records_info and pg_get_wal_records_info as a direct
c-function like in v8 patch [1]. A direct c-function always fares
better (84 msec vs 1400msec).

That indeed makes the view as is pretty much useless. And it'd probably be
worse in a workload with longer records / many FPIs.

+void
+_PG_init(void)

+void
+_PG_fini(void)

Why have this stuff if it's not used?

I kept it as a placeholder for future code additions, for instance
test_decoding.c and ssl_passphrase_func.c has empty _PG_init(),
_PG_fini(). If okay, I can mention there like "placeholder for now",
otherwise I can remove it.

That's not comparable, the test_decoding case has it as a placeholder because
it serves as a template to create further output plugins. Something not the
case here. So please remove.

+     for (block_id = 0; block_id <= record->max_block_id; block_id++)
+     {

To me duplicating this much code from waldump seems like a bad idea from a
maintainability POV.

Even if we were to put the above code from pg_walinspect and
pg_waldump into, say, walutils.c or some other existing file, there we
had to make if (pg_walinspect) appendStringInfo else if (pg_waldump)
printf() sort of thing, isn't it clumsy?

Why is that needed? Just use the stringinfo in both places? You're outputting
the exact same thing in both places right now. There's already a stringinfo in
XLogDumpDisplayRecord() these days (there wasn't back when pg_xlogddump was
written), so you could just convert at least the relevant printfs in
XLogDumpDisplayRecord().

Also, unnecessary if
conditions need to be executed for every record. For maintainability,
I added a note in pg_walinspect.c and pg_waldump.c to consider fixing
things in both places (of course this might sound dumbest way of doing
it, IMHO, it's sensible, given the if(pg_walinspect)-else
if(pg_waldump) sorts of code that we need to do in the common
functions). Thoughts?

IMO we shouldn't merge this with as much duplication as there is right now,
the notes don't change that it's a PITA to maintain.

Yeah. It is to handle some edge cases to print the WAL upto end_lsn
and avoid waiting in read_local_xlog_page. I will change it.

Actually, there's an open point as specified in [3]. Any thoughts on it?

Seems more user-friendly to wait - it's otherwise hard for a user to know what
LSN to put in.

Greetings,

Andres Freund

#72Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Andres Freund (#71)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

At Tue, 22 Mar 2022 11:00:06 -0700, Andres Freund <andres@anarazel.de> wrote in

Hi,

On 2022-03-22 21:57:51 +0530, Bharath Rupireddy wrote:

This is probably close to an order of magnitude slower than pg_waldump
--stats. Which imo renders this largely useless.

Yeah that's true. Do you suggest having pg_get_wal_stats() a
c-function like in v8 patch [1]?

Yes.

SEe some numbers at [2] with pg_get_wal_stats using
pg_get_wal_records_info and pg_get_wal_records_info as a direct
c-function like in v8 patch [1]. A direct c-function always fares
better (84 msec vs 1400msec).

That indeed makes the view as is pretty much useless. And it'd probably be
worse in a workload with longer records / many FPIs.

FWIW agreed. The SQL version is too slow..

+     for (block_id = 0; block_id <= record->max_block_id; block_id++)
+     {

To me duplicating this much code from waldump seems like a bad idea from a
maintainability POV.

Even if we were to put the above code from pg_walinspect and
pg_waldump into, say, walutils.c or some other existing file, there we
had to make if (pg_walinspect) appendStringInfo else if (pg_waldump)
printf() sort of thing, isn't it clumsy?

Why is that needed? Just use the stringinfo in both places? You're outputting
the exact same thing in both places right now. There's already a stringinfo in
XLogDumpDisplayRecord() these days (there wasn't back when pg_xlogddump was
written), so you could just convert at least the relevant printfs in
XLogDumpDisplayRecord().

Also, unnecessary if
conditions need to be executed for every record. For maintainability,
I added a note in pg_walinspect.c and pg_waldump.c to consider fixing
things in both places (of course this might sound dumbest way of doing
it, IMHO, it's sensible, given the if(pg_walinspect)-else
if(pg_waldump) sorts of code that we need to do in the common
functions). Thoughts?

IMO we shouldn't merge this with as much duplication as there is right now,
the notes don't change that it's a PITA to maintain.

The two places emit different outputs but the only difference is the
delimiter between two blockrefs. (By the way, the current code forgets
to insert a delimiter there). So even if the function took "bool
is_waldump", it is used only when appending a line delimiter. It
would be nicer if the "bool is_waldump" were "char *delimiter".
Others might think differently, though..

So, the function looks like this.

StringInfo XLogBlockRefInfos(XLogReaderState *record, char *delimiter,
uint32 &fpi_len);

Yeah. It is to handle some edge cases to print the WAL upto end_lsn
and avoid waiting in read_local_xlog_page. I will change it.

Actually, there's an open point as specified in [3]. Any thoughts on it?

Seems more user-friendly to wait - it's otherwise hard for a user to know what
LSN to put in.

I'm not sure it is user-friendly that the function "freeze"s for a
reason uncertain to the user.. Even if any results are accumulated
before waiting, all of them vanishes by entering Ctrl-C to release the
"freeze".

About the usefulness of the waiting behavior, it depends on what we
think the function's major use cases are. Robert (AFAIU) thinks it as
a simple WAL dumper that is intended to use in some automated
mechanism. The start/end LSNs simply identifies the records to emit.
No warning/errors and no waits except for apparently invalid inputs.

I thought it as a means by which to manually inspect wal on SQL
interface but don't have a strong opinion on the waiting behavior.
(Because I can avoid that by giving a valid LSN pair to the function
if I don't want it to "freeze".)

Anyway, the opinions here on the interface are described as follows.

A. as a diag interface for human use.

1. If the whole region is filled with records, return them all.
2. If start-LSN is too past, starts from the first available record.

3-1. If start-LSN is in futnre, wait for the record to come.
4-1. If end-LSN is in future, waits for new records.
5-1. If end-LSN is too past, error out?

B. as a simple WAL dumper

1. If the whole region is filled with records, return them all.
2. If start-LSN is too past, starts from the first available record.

3-2. If start-LSN is in futnre, returns nothig.
4-2. If end-LSN is in future, ends with the last available record.
5-2. If end-LSN is too past, returns northing.

1 and 2 are uncontroversial.

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#73Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Kyotaro Horiguchi (#72)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

At Wed, 23 Mar 2022 11:51:25 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

The two places emit different outputs but the only difference is the
delimiter between two blockrefs. (By the way, the current code forgets
to insert a delimiter there). So even if the function took "bool
is_waldump", it is used only when appending a line delimiter. It
would be nicer if the "bool is_waldump" were "char *delimiter".
Others might think differently, though..

So, the function looks like this.

StringInfo XLogBlockRefInfos(XLogReaderState *record, char *delimiter,
uint32 &fpi_len);

By the way, xlog_block_info@xlogrecovery.c has the subset of the
function. So the function can be shared with the callers of
xlog_block_info but I'm not sure it is not too-much...

StringInfo XLogBlockRefInfos(XLogReaderState *record, char *delimiter,
bool fpw_detail, uint32 &fpi_len);

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#74Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Andres Freund (#71)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Tue, Mar 22, 2022 at 11:30 PM Andres Freund <andres@anarazel.de> wrote:

Why "CALLED ON NULL INPUT"? It doesn't make sense to call the function with a
NULL lsn, does it? Also, that's the default, why is it restated here?

pg_get_wal_records_info needed that option (if end_lsn being the
default, providing WAL info upto the end of WAL). Also, we can emit
better error message ("invalid WAL start LSN") instead of generic one.
I wanted to keep error message and code same across all the functions
hence CALLED ON NULL INPUT option for pg_get_raw_wal_record.

I think it should be strict if it behaves strict. I fail to see what
consistency in error messages is worth here. And I'd probably just create two
different functions for begin and begin & end LSN and mark those as strict as
well.

I'm okay with changing them to be STRICT. Right now, the behaviour of
pg_get_wal_records_info is this:
CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
IN end_lsn pg_lsn DEFAULT NULL,

select pg_get_wal_records_info(start_lsn, end_lsn);
if start_lsn is future, then errors out
if end_lsn is future, then errors out
otherwise, returns WAL records info between start_lsn and end_lsn

select pg_get_wal_records_info(start_lsn);
if start_lsn is future, then errors out
sets end_lsn = current server lsn and returns WAL records info between
start_lsn and end_lsn

Same is true for pg_get_wal_stats.

Getting WAL records info provided start_lsn until end-of-WAL is a
basic ask and a good function to have. Now, if I were to make
pg_get_wal_records_info STRICT, then I would need to have another
function like pg_get_wal_records_info_till_end_of_wal/pg_get_wal_stats_till_end_of_wal
much like ones in few of my initial patches upthread.

Is it okay to have these functions pg_get_wal_records_info(start_lsn,
end_lsn)/pg_get_wal_stats(start_lsn, end_lsn) and
pg_get_wal_records_info_till_end_of_wal(start_lsn)/pg_get_wal_stats_till_end_of_wal(start_lsn)?
This way, it will be more clear to the user actually than to stuff
more than one behaviour in a single function with default values.

Please let me know your thoughts.

Yeah. It is to handle some edge cases to print the WAL upto end_lsn
and avoid waiting in read_local_xlog_page. I will change it.

Actually, there's an open point as specified in [3]. Any thoughts on it?

Seems more user-friendly to wait - it's otherwise hard for a user to know what
LSN to put in.

I agree with Kyotaro-san that the wait behavior isn't a good choice,
because CTRL+C would not emit the accumulated info/stats unlike
pg_waldump. Also, with wait behaviour it's easy for a user to trick
the server with an unreasonably futuristic WAL LSN, say F/FFFFFFFF.
Also, if we use pg_walinspect functions, say, within a WAL monitoring
app, the wait behaviour isn't good there as it might look like the
functions hanging the app. We might think about adding a timeout for
waiting, but that doesn't seem an elegant way. Users/Apps can easily
figure out the LSNs to get WAL info/stats - either they can use
pg_current_wal_XXXX or by looking at the control file or server logs
or pg_stat_replication, what not. LSNs are everywhere within the
postgres eco-system.

Instead, the functions simply can figure out what's current server LSN
at-the-moment and choose to error out if any of the provided input LSN
is beyond that as it's being done currently. This looks simpler and
user-friendly.

On Wed, Mar 23, 2022 at 8:27 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

At Wed, 23 Mar 2022 11:51:25 +0900 (JST), Kyotaro Horiguchi <horikyota.ntt@gmail.com> wrote in

The two places emit different outputs but the only difference is the
delimiter between two blockrefs. (By the way, the current code forgets
to insert a delimiter there). So even if the function took "bool
is_waldump", it is used only when appending a line delimiter. It
would be nicer if the "bool is_waldump" were "char *delimiter".
Others might think differently, though..

So, the function looks like this.

StringInfo XLogBlockRefInfos(XLogReaderState *record, char *delimiter,
uint32 &fpi_len);

By the way, xlog_block_info@xlogrecovery.c has the subset of the
function. So the function can be shared with the callers of
xlog_block_info but I'm not sure it is not too-much...

StringInfo XLogBlockRefInfos(XLogReaderState *record, char *delimiter,
bool fpw_detail, uint32 &fpi_len);

Yes, putting them in a common function is a good idea. I'm thinking
something like below.
StringInfo
XLogBlockRefInfos(XLogReaderState *record, char *delimiter,
uint32 *fpi_len, bool detailed_format)

I will try to put the common functions in xlogreader.h/.c, so that
both pg_waldump and pg_walinspect can make use of it. Thoughts?

Regards,
Bharath Rupireddy.

#75Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Andres Freund (#71)
1 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Tue, Mar 22, 2022 at 11:30 PM Andres Freund <andres@anarazel.de> wrote:

To me duplicating this much code from waldump seems like a bad idea from a
maintainability POV.

Even if we were to put the above code from pg_walinspect and
pg_waldump into, say, walutils.c or some other existing file, there we
had to make if (pg_walinspect) appendStringInfo else if (pg_waldump)
printf() sort of thing, isn't it clumsy?

Why is that needed? Just use the stringinfo in both places? You're outputting
the exact same thing in both places right now. There's already a stringinfo in
XLogDumpDisplayRecord() these days (there wasn't back when pg_xlogddump was
written), so you could just convert at least the relevant printfs in
XLogDumpDisplayRecord().

Also, unnecessary if
conditions need to be executed for every record. For maintainability,
I added a note in pg_walinspect.c and pg_waldump.c to consider fixing
things in both places (of course this might sound dumbest way of doing
it, IMHO, it's sensible, given the if(pg_walinspect)-else
if(pg_waldump) sorts of code that we need to do in the common
functions). Thoughts?

IMO we shouldn't merge this with as much duplication as there is right now,
the notes don't change that it's a PITA to maintain.

Here's a refactoring patch that basically moves the pg_waldump's
functions and stats structures to xlogreader.h/.c so that the
pg_walinspect can reuse them. If it looks okay, I will send the
pg_walinspect patches based on it.

Regards,
Bharath Rupireddy.

Attachments:

v13-0001-Refactor-pg_waldump-code.patchapplication/octet-stream; name=v13-0001-Refactor-pg_waldump-code.patchDownload
From 79e5f8d77140f26e1c27924aef1cbb14a89eb5d6 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 23 Mar 2022 15:49:03 +0000
Subject: [PATCH v13] Refactor pg_waldump code

This patch puts some generic chunks of pg_waldump's code
into separate reusable functions in xlogreader.h/.c.
---
 src/backend/access/transam/xlogreader.c | 197 +++++++++++++++++++++
 src/bin/pg_waldump/pg_waldump.c         | 217 ++----------------------
 src/common/relpath.c                    |  17 ++
 src/include/access/xlogreader.h         |  49 +++++-
 src/include/common/relpath.h            |   1 +
 5 files changed, 280 insertions(+), 201 deletions(-)

diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index e437c42992..cf31df7e24 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -1950,6 +1950,203 @@ XLogRecGetBlockTag(XLogReaderState *record, uint8 block_id,
 	return true;
 }
 
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+void
+XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+						  uint32 *fpi_len)
+{
+	int			block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Returns a string giving information about all the blocks in an
+ * XLogRecord.
+ */
+void
+XLogRecGetBlockRefInfo(XLogReaderState *record, char *delimiter,
+					   uint32 *fpi_len, bool detailed_format,
+					   StringInfo buf)
+{
+	RelFileNode rnode;
+	ForkNumber	forknum;
+	BlockNumber blk;
+	int	block_id;
+
+	if (detailed_format && delimiter)
+		appendStringInfoChar(buf, '\n');
+
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (detailed_format)
+		{
+			/* Get block references in detailed format. */
+
+			appendStringInfo(buf,
+							 "\tblkref #%d: rel %u/%u/%u fork %s blk %u",
+							 block_id,
+							 rnode.spcNode, rnode.dbNode, rnode.relNode,
+							 get_forkname(forknum),
+							 blk);
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
+
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (BKPIMAGE_COMPRESSED(bimg_info))
+				{
+					const char *method;
+
+					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+						method = "pglz";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+						method = "lz4";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
+						method = "zstd";
+					else
+						method = "unknown";
+
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u, "
+									 "compression saved: %u, method: %s",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length,
+									 BLCKSZ -
+									 XLogRecGetBlock(record, block_id)->hole_length -
+									 XLogRecGetBlock(record, block_id)->bimg_len,
+									 method);
+				}
+				else
+				{
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length);
+				}
+			}
+		}
+		else
+		{
+			/* Get block references in short format. */
+
+			if (forknum != MAIN_FORKNUM)
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u fork %s blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 get_forkname(forknum),
+								 blk);
+			}
+			else
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 blk);
+			}
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (XLogRecBlockImageApply(record, block_id))
+					appendStringInfo(buf, " FPW");
+				else
+					appendStringInfo(buf, " FPW for WAL verification");
+			}
+		}
+
+		if (detailed_format && delimiter)
+			appendStringInfoChar(buf, '\n');
+	}
+
+	if (!detailed_format && delimiter)
+		appendStringInfoChar(buf, '\n');
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+void
+XLogRecStoreStats(XLogStats *stats, XLogReaderState *record)
+{
+	RmgrId		rmid;
+	uint8		recid;
+	uint32		rec_len;
+	uint32		fpi_len;
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	XLogRecGetLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics */
+
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
 /*
  * Returns the data associated with a block reference, or NULL if there is
  * no data (e.g. because a full-page image was taken instead). The returned
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index fc081adfb8..03bc22c0f9 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -57,23 +57,6 @@ typedef struct XLogDumpConfig
 	bool		filter_by_xid_enabled;
 } XLogDumpConfig;
 
-typedef struct Stats
-{
-	uint64		count;
-	uint64		rec_len;
-	uint64		fpi_len;
-} Stats;
-
-#define MAX_XLINFO_TYPES 16
-
-typedef struct XLogDumpStats
-{
-	uint64		count;
-	XLogRecPtr	startptr;
-	XLogRecPtr	endptr;
-	Stats		rmgr_stats[RM_NEXT_ID];
-	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
-} XLogDumpStats;
 
 #define fatal_error(...) do { pg_log_fatal(__VA_ARGS__); exit(EXIT_FAILURE); } while(0)
 
@@ -391,81 +374,6 @@ WALDumpReadPage(XLogReaderState *state, XLogRecPtr targetPagePtr, int reqLen,
 	return count;
 }
 
-/*
- * Calculate the size of a record, split into !FPI and FPI parts.
- */
-static void
-XLogDumpRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
-{
-	int			block_id;
-
-	/*
-	 * Calculate the amount of FPI data in the record.
-	 *
-	 * XXX: We peek into xlogreader's private decoded backup blocks for the
-	 * bimg_len indicating the length of FPI data.
-	 */
-	*fpi_len = 0;
-	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-	{
-		if (XLogRecHasBlockImage(record, block_id))
-			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
-	}
-
-	/*
-	 * Calculate the length of the record as the total length - the length of
-	 * all the block images.
-	 */
-	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
-}
-
-/*
- * Store per-rmgr and per-record statistics for a given record.
- */
-static void
-XLogDumpCountRecord(XLogDumpConfig *config, XLogDumpStats *stats,
-					XLogReaderState *record)
-{
-	RmgrId		rmid;
-	uint8		recid;
-	uint32		rec_len;
-	uint32		fpi_len;
-
-	stats->count++;
-
-	rmid = XLogRecGetRmid(record);
-
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
-
-	/* Update per-rmgr statistics */
-
-	stats->rmgr_stats[rmid].count++;
-	stats->rmgr_stats[rmid].rec_len += rec_len;
-	stats->rmgr_stats[rmid].fpi_len += fpi_len;
-
-	/*
-	 * Update per-record statistics, where the record is identified by a
-	 * combination of the RmgrId and the four bits of the xl_info field that
-	 * are the rmgr's domain (resulting in sixteen possible entries per
-	 * RmgrId).
-	 */
-
-	recid = XLogRecGetInfo(record) >> 4;
-
-	/*
-	 * XACT records need to be handled differently. Those records use the
-	 * first bit of those four bits for an optional flag variable and the
-	 * following three bits for the opcode. We filter opcode out of xl_info
-	 * and use it as the identifier of the record.
-	 */
-	if (rmid == RM_XACT_ID)
-		recid &= 0x07;
-
-	stats->record_stats[rmid][recid].count++;
-	stats->record_stats[rmid][recid].rec_len += rec_len;
-	stats->record_stats[rmid][recid].fpi_len += fpi_len;
-}
-
 /*
  * Print a record to stdout
  */
@@ -476,15 +384,13 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	const RmgrDescData *desc = &RmgrDescTable[XLogRecGetRmid(record)];
 	uint32		rec_len;
 	uint32		fpi_len;
-	RelFileNode rnode;
-	ForkNumber	forknum;
-	BlockNumber blk;
-	int			block_id;
 	uint8		info = XLogRecGetInfo(record);
 	XLogRecPtr	xl_prev = XLogRecGetPrev(record);
 	StringInfoData s;
+	StringInfoData	blk_ref;
+	char	delim = {'\n'};
 
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
+	XLogRecGetLen(record, &rec_len, &fpi_len);
 
 	printf("rmgr: %-11s len (rec/tot): %6u/%6u, tx: %10u, lsn: %X/%08X, prev %X/%08X, ",
 		   desc->rm_name,
@@ -504,91 +410,12 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	printf("%s", s.data);
 	pfree(s.data);
 
-	if (!config->bkp_details)
-	{
-		/* print block references (short format) */
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
+	initStringInfo(&blk_ref);
+	XLogRecGetBlockRefInfo(record, &delim, NULL, config->bkp_details,
+						   &blk_ref);
 
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			if (forknum != MAIN_FORKNUM)
-				printf(", blkref #%d: rel %u/%u/%u fork %s blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   forkNames[forknum],
-					   blk);
-			else
-				printf(", blkref #%d: rel %u/%u/%u blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				if (XLogRecBlockImageApply(record, block_id))
-					printf(" FPW");
-				else
-					printf(" FPW for WAL verification");
-			}
-		}
-		putchar('\n');
-	}
-	else
-	{
-		/* print block references (detailed format) */
-		putchar('\n');
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			printf("\tblkref #%d: rel %u/%u/%u fork %s blk %u",
-				   block_id,
-				   rnode.spcNode, rnode.dbNode, rnode.relNode,
-				   forkNames[forknum],
-				   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
-
-				if (BKPIMAGE_COMPRESSED(bimg_info))
-				{
-					const char *method;
-
-					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
-						method = "pglz";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
-						method = "lz4";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
-						method = "zstd";
-					else
-						method = "unknown";
-
-					printf(" (FPW%s); hole: offset: %u, length: %u, "
-						   "compression saved: %u, method: %s",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length,
-						   BLCKSZ -
-						   XLogRecGetBlock(record, block_id)->hole_length -
-						   XLogRecGetBlock(record, block_id)->bimg_len,
-						   method);
-				}
-				else
-				{
-					printf(" (FPW%s); hole: offset: %u, length: %u",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length);
-				}
-			}
-			putchar('\n');
-		}
-	}
+	printf("%s", blk_ref.data);
+	pfree(blk_ref.data);
 }
 
 /*
@@ -606,21 +433,11 @@ XLogDumpStatsRow(const char *name,
 				fpi_len_pct,
 				tot_len_pct;
 
-	n_pct = 0;
-	if (total_count != 0)
-		n_pct = 100 * (double) n / total_count;
-
-	rec_len_pct = 0;
-	if (total_rec_len != 0)
-		rec_len_pct = 100 * (double) rec_len / total_rec_len;
-
-	fpi_len_pct = 0;
-	if (total_fpi_len != 0)
-		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
-
-	tot_len_pct = 0;
-	if (total_len != 0)
-		tot_len_pct = 100 * (double) tot_len / total_len;
+	XLOG_GET_STATS_PERCENTAGE(n_pct, rec_len_pct, fpi_len_pct,
+							  tot_len_pct, total_count,
+							  rec_len, total_rec_len,
+							  fpi_len, total_fpi_len,
+							  tot_len, total_len);
 
 	printf("%-27s "
 		   "%20" INT64_MODIFIER "u (%6.02f) "
@@ -636,7 +453,7 @@ XLogDumpStatsRow(const char *name,
  * Display summary statistics about the records seen so far.
  */
 static void
-XLogDumpDisplayStats(XLogDumpConfig *config, XLogDumpStats *stats)
+XLogDumpDisplayStats(XLogDumpConfig *config, XLogStats *stats)
 {
 	int			ri,
 				rj;
@@ -792,7 +609,7 @@ main(int argc, char **argv)
 	XLogReaderState *xlogreader_state;
 	XLogDumpPrivate private;
 	XLogDumpConfig config;
-	XLogDumpStats stats;
+	XLogStats stats;
 	XLogRecord *record;
 	XLogRecPtr	first_record;
 	char	   *waldir = NULL;
@@ -842,7 +659,7 @@ main(int argc, char **argv)
 
 	memset(&private, 0, sizeof(XLogDumpPrivate));
 	memset(&config, 0, sizeof(XLogDumpConfig));
-	memset(&stats, 0, sizeof(XLogDumpStats));
+	memset(&stats, 0, sizeof(XLogStats));
 
 	private.timeline = 1;
 	private.startptr = InvalidXLogRecPtr;
@@ -1153,7 +970,7 @@ main(int argc, char **argv)
 		{
 			if (config.stats == true)
 			{
-				XLogDumpCountRecord(&config, &stats, xlogreader_state);
+				XLogRecStoreStats(&stats, xlogreader_state);
 				stats.endptr = xlogreader_state->EndRecPtr;
 			}
 			else
diff --git a/src/common/relpath.c b/src/common/relpath.c
index 636c96efd3..00bf1f37d5 100644
--- a/src/common/relpath.c
+++ b/src/common/relpath.c
@@ -39,6 +39,23 @@ const char *const forkNames[] = {
 
 StaticAssertDecl(lengthof(forkNames) == (MAX_FORKNUM + 1),
 				 "array length mismatch");
+/*
+ * get_forkname - return fork name given fork number
+ *
+ * This function is defined with "extern PGDLLIMPORT ..." in the core here so
+ * that the loadable modules can access it.
+ */
+const char *const
+get_forkname(ForkNumber num)
+{
+	/*
+	 * As this function gets called by external modules, let's ensure that the
+	 * fork number passed in is valid.
+	 */
+	Assert(num > InvalidForkNumber && num <= MAX_FORKNUM);
+
+	return forkNames[num];
+}
 
 /*
  * forkname_to_number - look up fork number by name
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index f4388cc9be..c9acbfe051 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -39,6 +39,7 @@
 #endif
 
 #include "access/xlogrecord.h"
+#include "lib/stringinfo.h"
 
 /* WALOpenSegment represents a WAL segment being read. */
 typedef struct WALOpenSegment
@@ -314,6 +315,46 @@ struct XLogReaderState
 	bool		nonblocking;
 };
 
+#define MAX_XLINFO_TYPES 16
+
+#define XLOG_GET_STATS_PERCENTAGE(n_pct, rec_len_pct, fpi_len_pct, \
+								  tot_len_pct, total_count, \
+								  rec_len, total_rec_len, \
+								  fpi_len, total_fpi_len, \
+								  tot_len, total_len) \
+do { \
+	n_pct = 0; \
+	if (total_count != 0) \
+		n_pct = 100 * (double) n / total_count; \
+	rec_len_pct = 0; \
+	if (total_rec_len != 0) \
+		rec_len_pct = 100 * (double) rec_len / total_rec_len; \
+	fpi_len_pct = 0; \
+	if (total_fpi_len != 0) \
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len; \
+	tot_len_pct = 0; \
+	if (total_len != 0) \
+		tot_len_pct = 100 * (double) tot_len / total_len; \
+} while(0)
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+} XLogRecStats;
+
+typedef struct XLogStats
+{
+	uint64		count;
+#ifdef FRONTEND
+	XLogRecPtr	startptr;
+	XLogRecPtr	endptr;
+#endif
+	XLogRecStats	rmgr_stats[RM_NEXT_ID];
+	XLogRecStats	record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+} XLogStats;
+
 /*
  * Check if XLogNextRecord() has any more queued records or an error to return.
  */
@@ -430,5 +471,11 @@ extern char *XLogRecGetBlockData(XLogReaderState *record, uint8 block_id, Size *
 extern bool XLogRecGetBlockTag(XLogReaderState *record, uint8 block_id,
 							   RelFileNode *rnode, ForkNumber *forknum,
 							   BlockNumber *blknum);
-
+extern void XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+						  uint32 *fpi_len);
+extern void XLogRecGetBlockRefInfo(XLogReaderState *record,
+								   char *delimiter, uint32 *fpi_len,
+								   bool detailed_format,
+								   StringInfo blk_ref);
+extern void XLogRecStoreStats(XLogStats *stats, XLogReaderState *record);
 #endif							/* XLOGREADER_H */
diff --git a/src/include/common/relpath.h b/src/include/common/relpath.h
index a4b5dc853b..072d381d5f 100644
--- a/src/include/common/relpath.h
+++ b/src/include/common/relpath.h
@@ -58,6 +58,7 @@ typedef enum ForkNumber
 
 extern const char *const forkNames[];
 
+extern PGDLLIMPORT const char *const get_forkname(ForkNumber num);
 extern ForkNumber forkname_to_number(const char *forkName);
 extern int	forkname_chars(const char *str, ForkNumber *fork);
 
-- 
2.25.1

#76Kyotaro Horiguchi
horikyota.ntt@gmail.com
In reply to: Bharath Rupireddy (#75)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

At Wed, 23 Mar 2022 21:36:09 +0530, Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com> wrote in

Here's a refactoring patch that basically moves the pg_waldump's
functions and stats structures to xlogreader.h/.c so that the
pg_walinspect can reuse them. If it looks okay, I will send the
pg_walinspect patches based on it.

+void
+XLogRecGetBlockRefInfo(XLogReaderState *record, char *delimiter,
+					   uint32 *fpi_len, bool detailed_format,
+					   StringInfo buf)
...
+		if (detailed_format && delimiter)
+			appendStringInfoChar(buf, '\n');

It is odd that the variable "delimiter" is used as a bool in the
function, though it is a "char *", which I meant that it is used as
delimiter string (assuming that you might want to insert ", " between
two blkref descriptions).

+get_forkname(ForkNumber num)

forkNames[] is public and used in reinit.c. I think we don't need
this function.

+#define MAX_XLINFO_TYPES 16
...
+	XLogRecStats	rmgr_stats[RM_NEXT_ID];
+	XLogRecStats	record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+} XLogStats;
+

This doesn't seem to be a part of xlogreader. Couldn't we add a new
module "xlogstats"? XLogRecGetBlockRefInfo also doesn't seem to me as
a part of xlogreader, the xlogstats looks like a better place.

+#define XLOG_GET_STATS_PERCENTAGE(n_pct, rec_len_pct, fpi_len_pct, \
+								  tot_len_pct, total_count, \

It doesn't need to be a macro. However in the first place I don't
think it is useful to have. Rather it may be harmful since it doesn't
reduce complexity much but instead just hides details. If we want to
avoid tedious repetitions of the same statements, a macro like the
following may work.

#define CALC_PCT (num, denom) ((denom) == 0 ? 0.0 ? 100.0 * (num) / (denom))
...

n_pct = CALC_PCT(n, total_count);
rec_len_pct = CALC_PCT(rec_len, total_rec_len);
fpi_len_pct = CALC_PCT(fpi_len, total_fpi_len);
tot_len_pct = CALC_PCT(tot_len, total_len);

But it is not seem that different if we directly write out the detail.

n_pct = (total_count == 0 ? 0 : 100.0 * n / total_count);
rec_len_pct = (total_rec_len == 0 ? 0 : 100.0 * rec_len / total_rec_len);
fpi_len_pct = (total_fpi_len == 0 ? 0 : 100.0 * fpi_len / total_fpi_len);
tot_len_pct = (total_len == 0 ? 0 : 100.0 * tot_len / total_len);

regards.

--
Kyotaro Horiguchi
NTT Open Source Software Center

#77Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Kyotaro Horiguchi (#76)
4 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Mar 24, 2022 at 10:22 AM Kyotaro Horiguchi
<horikyota.ntt@gmail.com> wrote:

+void
+XLogRecGetBlockRefInfo(XLogReaderState *record, char *delimiter,
+                                          uint32 *fpi_len, bool detailed_format,
+                                          StringInfo buf)
...
+               if (detailed_format && delimiter)
+                       appendStringInfoChar(buf, '\n');

It is odd that the variable "delimiter" is used as a bool in the
function, though it is a "char *", which I meant that it is used as
delimiter string (assuming that you might want to insert ", " between
two blkref descriptions).

I'm passing NULL if the delimiter isn't required (for pg_walinspect)
and I wanted to check if it's passed, so I was using the delimiter in
the condition. However, I now changed it to delimiter != NULL.

+get_forkname(ForkNumber num)

forkNames[] is public and used in reinit.c. I think we don't need
this function.

Yes. I removed it.

+#define MAX_XLINFO_TYPES 16
...
+       XLogRecStats    rmgr_stats[RM_NEXT_ID];
+       XLogRecStats    record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+} XLogStats;
+

This doesn't seem to be a part of xlogreader. Couldn't we add a new
module "xlogstats"? XLogRecGetBlockRefInfo also doesn't seem to me as
a part of xlogreader, the xlogstats looks like a better place.

I'm not sure if it's worth adding new files xlogstats.h/.c just for 2
structures, 1 macro, and 2 functions with no plan to add new stats
structures or functions. Since xlogreader is the one that reads the
WAL, and is being included by both backend and other modules (tools
and extensions) IMO it's the right place. However, I can specify in
xlogreader that if at all the new stats related structures or
functions are going to be added, it's good to move them into a new
header and .c file.

Thoughts?

+#define XLOG_GET_STATS_PERCENTAGE(n_pct, rec_len_pct, fpi_len_pct, \
+                                                                 tot_len_pct, total_count, \

It doesn't need to be a macro. However in the first place I don't
think it is useful to have. Rather it may be harmful since it doesn't
reduce complexity much but instead just hides details. If we want to
avoid tedious repetitions of the same statements, a macro like the
following may work.

#define CALC_PCT (num, denom) ((denom) == 0 ? 0.0 ? 100.0 * (num) / (denom))
...

n_pct = CALC_PCT(n, total_count);
rec_len_pct = CALC_PCT(rec_len, total_rec_len);
fpi_len_pct = CALC_PCT(fpi_len, total_fpi_len);
tot_len_pct = CALC_PCT(tot_len, total_len);

But it is not seem that different if we directly write out the detail.

n_pct = (total_count == 0 ? 0 : 100.0 * n / total_count);
rec_len_pct = (total_rec_len == 0 ? 0 : 100.0 * rec_len / total_rec_len);
fpi_len_pct = (total_fpi_len == 0 ? 0 : 100.0 * fpi_len / total_fpi_len);
tot_len_pct = (total_len == 0 ? 0 : 100.0 * tot_len / total_len);

I removed the XLOG_GET_STATS_PERCENTAGE macro.

Attaching v14 patch-set here with. It has bunch of other changes along
with the above:

1) Used STRICT for all the functions and introduced _till_end_of_wal
versions for pg_get_wal_records_info and pg_get_wal_stats.
2) Most of the code is reused between pg_walinspect and pg_waldump and
also within pg_walinspect.
3) Added read_local_xlog_page_no_wait without duplicating the code so
that the pg_walinspect functions don't wait even while finding the
first valid WAL record.
4) No function waits for future WAL lsn even to find the first valid WAL record.
5) Addressed the review comments raised upthread by Andres.

I hope this version makes the patch cleaner, please review it further.

Regards,
Bharath Rupireddy.

Attachments:

v14-0001-Refactor-pg_waldump-code.patchapplication/octet-stream; name=v14-0001-Refactor-pg_waldump-code.patchDownload
From f38e8ee17cf2b6e413ae6c2ffd8400c26457180d Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 24 Mar 2022 09:23:15 +0000
Subject: [PATCH v14] Refactor pg_waldump code

This patch puts some generic chunks of pg_waldump's code
into separate reusable functions in xlogreader.h/.c.
---
 src/backend/access/transam/xlogreader.c | 202 ++++++++++++++++++++++++
 src/bin/pg_waldump/pg_waldump.c         | 197 ++---------------------
 src/include/access/xlogreader.h         |  28 ++++
 3 files changed, 240 insertions(+), 187 deletions(-)

diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index e437c42992..5909ec180e 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -1950,6 +1950,208 @@ XLogRecGetBlockTag(XLogReaderState *record, uint8 block_id,
 	return true;
 }
 
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+void
+XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+			  uint32 *fpi_len)
+{
+	int	block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Returns a string giving information about all the blocks in an
+ * XLogRecord.
+ */
+void
+XLogRecGetBlockRefInfo(XLogReaderState *record, char *delimiter,
+					   uint32 *fpi_len, bool detailed_format,
+					   StringInfo buf)
+{
+	int	block_id;
+
+	Assert(record != NULL);
+
+	if (detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		RelFileNode rnode = {InvalidOid, InvalidOid, InvalidOid};
+		ForkNumber	forknum = InvalidForkNumber;
+		BlockNumber blk = InvalidBlockNumber;
+
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (detailed_format)
+		{
+			/* Get block references in detailed format. */
+
+			appendStringInfo(buf,
+							 "\tblkref #%d: rel %u/%u/%u fork %s blk %u",
+							 block_id,
+							 rnode.spcNode, rnode.dbNode, rnode.relNode,
+							 forkNames[forknum],
+							 blk);
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
+
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (BKPIMAGE_COMPRESSED(bimg_info))
+				{
+					const char *method;
+
+					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+						method = "pglz";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+						method = "lz4";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
+						method = "zstd";
+					else
+						method = "unknown";
+
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u, "
+									 "compression saved: %u, method: %s",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length,
+									 BLCKSZ -
+									 XLogRecGetBlock(record, block_id)->hole_length -
+									 XLogRecGetBlock(record, block_id)->bimg_len,
+									 method);
+				}
+				else
+				{
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length);
+				}
+			}
+		}
+		else
+		{
+			/* Get block references in short format. */
+
+			if (forknum != MAIN_FORKNUM)
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u fork %s blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 forkNames[forknum],
+								 blk);
+			}
+			else
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 blk);
+			}
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (XLogRecBlockImageApply(record, block_id))
+					appendStringInfo(buf, " FPW");
+				else
+					appendStringInfo(buf, " FPW for WAL verification");
+			}
+		}
+
+		if (detailed_format && delimiter != NULL)
+			appendStringInfoChar(buf, '\n');
+	}
+
+	if (!detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+void
+XLogRecStoreStats(XLogStats *stats, XLogReaderState *record)
+{
+	RmgrId	rmid;
+	uint8	recid;
+	uint32	rec_len;
+	uint32	fpi_len;
+
+	Assert(stats != NULL && record != NULL);
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	XLogRecGetLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics */
+
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
+
 /*
  * Returns the data associated with a block reference, or NULL if there is
  * no data (e.g. because a full-page image was taken instead). The returned
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 92238f30c9..179cd617e4 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -66,23 +66,6 @@ typedef struct XLogDumpConfig
 	bool		filter_by_fpw;
 } XLogDumpConfig;
 
-typedef struct Stats
-{
-	uint64		count;
-	uint64		rec_len;
-	uint64		fpi_len;
-} Stats;
-
-#define MAX_XLINFO_TYPES 16
-
-typedef struct XLogDumpStats
-{
-	uint64		count;
-	XLogRecPtr	startptr;
-	XLogRecPtr	endptr;
-	Stats		rmgr_stats[RM_NEXT_ID];
-	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
-} XLogDumpStats;
 
 #define fatal_error(...) do { pg_log_fatal(__VA_ARGS__); exit(EXIT_FAILURE); } while(0)
 
@@ -453,81 +436,6 @@ XLogRecordHasFPW(XLogReaderState *record)
 	return false;
 }
 
-/*
- * Calculate the size of a record, split into !FPI and FPI parts.
- */
-static void
-XLogDumpRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
-{
-	int			block_id;
-
-	/*
-	 * Calculate the amount of FPI data in the record.
-	 *
-	 * XXX: We peek into xlogreader's private decoded backup blocks for the
-	 * bimg_len indicating the length of FPI data.
-	 */
-	*fpi_len = 0;
-	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-	{
-		if (XLogRecHasBlockImage(record, block_id))
-			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
-	}
-
-	/*
-	 * Calculate the length of the record as the total length - the length of
-	 * all the block images.
-	 */
-	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
-}
-
-/*
- * Store per-rmgr and per-record statistics for a given record.
- */
-static void
-XLogDumpCountRecord(XLogDumpConfig *config, XLogDumpStats *stats,
-					XLogReaderState *record)
-{
-	RmgrId		rmid;
-	uint8		recid;
-	uint32		rec_len;
-	uint32		fpi_len;
-
-	stats->count++;
-
-	rmid = XLogRecGetRmid(record);
-
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
-
-	/* Update per-rmgr statistics */
-
-	stats->rmgr_stats[rmid].count++;
-	stats->rmgr_stats[rmid].rec_len += rec_len;
-	stats->rmgr_stats[rmid].fpi_len += fpi_len;
-
-	/*
-	 * Update per-record statistics, where the record is identified by a
-	 * combination of the RmgrId and the four bits of the xl_info field that
-	 * are the rmgr's domain (resulting in sixteen possible entries per
-	 * RmgrId).
-	 */
-
-	recid = XLogRecGetInfo(record) >> 4;
-
-	/*
-	 * XACT records need to be handled differently. Those records use the
-	 * first bit of those four bits for an optional flag variable and the
-	 * following three bits for the opcode. We filter opcode out of xl_info
-	 * and use it as the identifier of the record.
-	 */
-	if (rmid == RM_XACT_ID)
-		recid &= 0x07;
-
-	stats->record_stats[rmid][recid].count++;
-	stats->record_stats[rmid][recid].rec_len += rec_len;
-	stats->record_stats[rmid][recid].fpi_len += fpi_len;
-}
-
 /*
  * Print a record to stdout
  */
@@ -538,15 +446,12 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	const RmgrDescData *desc = &RmgrDescTable[XLogRecGetRmid(record)];
 	uint32		rec_len;
 	uint32		fpi_len;
-	RelFileNode rnode;
-	ForkNumber	forknum;
-	BlockNumber blk;
-	int			block_id;
 	uint8		info = XLogRecGetInfo(record);
 	XLogRecPtr	xl_prev = XLogRecGetPrev(record);
 	StringInfoData s;
+	char	delim = {'\n'};
 
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
+	XLogRecGetLen(record, &rec_len, &fpi_len);
 
 	printf("rmgr: %-11s len (rec/tot): %6u/%6u, tx: %10u, lsn: %X/%08X, prev %X/%08X, ",
 		   desc->rm_name,
@@ -564,93 +469,11 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	initStringInfo(&s);
 	desc->rm_desc(&s, record);
 	printf("%s", s.data);
-	pfree(s.data);
-
-	if (!config->bkp_details)
-	{
-		/* print block references (short format) */
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			if (forknum != MAIN_FORKNUM)
-				printf(", blkref #%d: rel %u/%u/%u fork %s blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   forkNames[forknum],
-					   blk);
-			else
-				printf(", blkref #%d: rel %u/%u/%u blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				if (XLogRecBlockImageApply(record, block_id))
-					printf(" FPW");
-				else
-					printf(" FPW for WAL verification");
-			}
-		}
-		putchar('\n');
-	}
-	else
-	{
-		/* print block references (detailed format) */
-		putchar('\n');
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			printf("\tblkref #%d: rel %u/%u/%u fork %s blk %u",
-				   block_id,
-				   rnode.spcNode, rnode.dbNode, rnode.relNode,
-				   forkNames[forknum],
-				   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
 
-				if (BKPIMAGE_COMPRESSED(bimg_info))
-				{
-					const char *method;
-
-					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
-						method = "pglz";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
-						method = "lz4";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
-						method = "zstd";
-					else
-						method = "unknown";
-
-					printf(" (FPW%s); hole: offset: %u, length: %u, "
-						   "compression saved: %u, method: %s",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length,
-						   BLCKSZ -
-						   XLogRecGetBlock(record, block_id)->hole_length -
-						   XLogRecGetBlock(record, block_id)->bimg_len,
-						   method);
-				}
-				else
-				{
-					printf(" (FPW%s); hole: offset: %u, length: %u",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length);
-				}
-			}
-			putchar('\n');
-		}
-	}
+	resetStringInfo(&s);
+	XLogRecGetBlockRefInfo(record, &delim, NULL, config->bkp_details, &s);
+	printf("%s", s.data);
+	pfree(s.data);
 }
 
 /*
@@ -698,7 +521,7 @@ XLogDumpStatsRow(const char *name,
  * Display summary statistics about the records seen so far.
  */
 static void
-XLogDumpDisplayStats(XLogDumpConfig *config, XLogDumpStats *stats)
+XLogDumpDisplayStats(XLogDumpConfig *config, XLogStats *stats)
 {
 	int			ri,
 				rj;
@@ -859,7 +682,7 @@ main(int argc, char **argv)
 	XLogReaderState *xlogreader_state;
 	XLogDumpPrivate private;
 	XLogDumpConfig config;
-	XLogDumpStats stats;
+	XLogStats stats;
 	XLogRecord *record;
 	XLogRecPtr	first_record;
 	char	   *waldir = NULL;
@@ -913,7 +736,7 @@ main(int argc, char **argv)
 
 	memset(&private, 0, sizeof(XLogDumpPrivate));
 	memset(&config, 0, sizeof(XLogDumpConfig));
-	memset(&stats, 0, sizeof(XLogDumpStats));
+	memset(&stats, 0, sizeof(XLogStats));
 
 	private.timeline = 1;
 	private.startptr = InvalidXLogRecPtr;
@@ -1295,7 +1118,7 @@ main(int argc, char **argv)
 		{
 			if (config.stats == true)
 			{
-				XLogDumpCountRecord(&config, &stats, xlogreader_state);
+				XLogRecStoreStats(&stats, xlogreader_state);
 				stats.endptr = xlogreader_state->EndRecPtr;
 			}
 			else
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index f4388cc9be..4021ae05df 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -39,6 +39,7 @@
 #endif
 
 #include "access/xlogrecord.h"
+#include "lib/stringinfo.h"
 
 /* WALOpenSegment represents a WAL segment being read. */
 typedef struct WALOpenSegment
@@ -314,6 +315,26 @@ struct XLogReaderState
 	bool		nonblocking;
 };
 
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64		count;
+	uint64		rec_len;
+	uint64		fpi_len;
+} XLogRecStats;
+
+typedef struct XLogStats
+{
+	uint64		count;
+#ifdef FRONTEND
+	XLogRecPtr	startptr;
+	XLogRecPtr	endptr;
+#endif
+	XLogRecStats	rmgr_stats[RM_NEXT_ID];
+	XLogRecStats	record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+} XLogStats;
+
 /*
  * Check if XLogNextRecord() has any more queued records or an error to return.
  */
@@ -430,5 +451,12 @@ extern char *XLogRecGetBlockData(XLogReaderState *record, uint8 block_id, Size *
 extern bool XLogRecGetBlockTag(XLogReaderState *record, uint8 block_id,
 							   RelFileNode *rnode, ForkNumber *forknum,
 							   BlockNumber *blknum);
+extern void XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+						  uint32 *fpi_len);
+extern void XLogRecGetBlockRefInfo(XLogReaderState *record,
+								   char *delimiter, uint32 *fpi_len,
+								   bool detailed_format,
+								   StringInfo blk_ref);
+extern void XLogRecStoreStats(XLogStats *stats, XLogReaderState *record);
 
 #endif							/* XLOGREADER_H */
-- 
2.25.1

v14-0002-pg_walinspect.patchapplication/octet-stream; name=v14-0002-pg_walinspect.patchDownload
From 302ccd68143ea08e9df83c76ae95a7d4d388ee8d Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 24 Mar 2022 09:24:57 +0000
Subject: [PATCH v14] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 132 ++++
 contrib/pg_walinspect/pg_walinspect.c        | 652 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  13 +-
 src/backend/access/transam/xlogutils.c       |  33 +
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/access/xlogutils.h               |   4 +
 13 files changed, 870 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 332b486ecc..bbf220407b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -41,6 +41,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..9b192dd2c1
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,132 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_wal_record()
+--
+CREATE FUNCTION pg_get_wal_record(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT record_length int4,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn) TO pg_read_server_files;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..20e43b403e
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,652 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(pg_get_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats_till_end_of_wal);
+
+typedef void (*GetWALDetailsCB) (FunctionCallInfo fcinfo,
+								 XLogRecPtr start_lsn,
+								 XLogRecPtr end_lsn);
+
+static bool IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static Datum GetWALRecordInternal(FunctionCallInfo fcinfo, Datum *values,
+								  bool *nulls, uint32 ncols, bool get_info);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+							  GetWALDetailsCB wal_details_cb);
+static void GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+							  XLogRecPtr end_lsn);
+static void GetXLogSummaryStats(XLogStats * stats, ReturnSetInfo *rsinfo,
+								Datum *values, bool *nulls, uint32 ncols);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						XLogRecPtr end_lsn);
+
+/*
+ * Determinte if the given LSN is in future and return the LSN up to which the
+ * server has WAL.
+ */
+static bool
+IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn)
+{
+	/*
+	 * We determine the current LSN of the server similar to how page_read
+	 * callback read_local_xlog_page_no_wait does.
+	 */
+	if (!RecoveryInProgress())
+		*curr_lsn = GetFlushRecPtr(NULL);
+	else
+		*curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(*curr_lsn));
+
+	if (lsn >= *curr_lsn)
+		return true;
+
+	return false;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page_no_wait,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	initStringInfo(&rec_desc);
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	desc->rm_desc(&rec_desc, record);
+
+	/* Block references. */
+	initStringInfo(&rec_blk_ref);
+	XLogRecGetBlockRefInfo(record, NULL, &fpi_len, true, &rec_blk_ref);
+
+	main_data_len = XLogRecGetDataLen(record);
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = PointerGetDatum(data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record data or info.
+ */
+Datum
+GetWALRecordInternal(FunctionCallInfo fcinfo, Datum *values, bool *nulls,
+					 uint32 ncols, bool get_info)
+{
+	XLogRecPtr	lsn;
+	XLogRecPtr	curr_lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	HeapTuple	tuple;
+	Datum	result;
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (IsFutureLSN(lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, ncols);
+	MemSet(nulls, 0, ncols);
+
+	if (get_info)
+	{
+		GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+						  ncols);
+	}
+	else
+	{
+		bytea	*rec;
+		uint32	rec_len;
+		char	*rec_data;
+		int	i = 0;
+
+		rec_len = XLogRecGetTotalLen(xlogreader);
+
+		Assert(rec_len > 0);
+
+		rec = (bytea *) palloc(rec_len + VARHDRSZ);
+		SET_VARSIZE(rec, rec_len + VARHDRSZ);
+		rec_data = VARDATA(rec);
+
+		memcpy(rec_data, record, rec_len);
+
+		values[i++] = LSNGetDatum(first_record);
+		values[i++] = LSNGetDatum(xlogreader->EndRecPtr - 1);
+		values[i++] = LSNGetDatum(XLogRecGetPrev(xlogreader));
+		values[i++] = UInt32GetDatum(rec_len);
+		values[i++] = PointerGetDatum(rec);
+
+		Assert(i == ncols);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	return result;
+}
+
+/*
+ * Get WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_COLS 5
+	Datum	result;
+	Datum	values[PG_GET_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_COLS];
+
+	result = GetWALRecordInternal(fcinfo, values, nulls,
+								  PG_GET_WAL_RECORD_COLS,
+								  false);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_COLS
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	Datum	result;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	result = GetWALRecordInternal(fcinfo, values, nulls,
+								  PG_GET_WAL_RECORD_INFO_COLS,
+								  true);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get WAL details such as record info, stats using the passed in callback.
+ */
+static void
+GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+				  GetWALDetailsCB wal_details_cb)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecPtr	curr_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* If not till end of wal, end_lsn would have been specified. */
+	if (!till_end_of_wal)
+		end_lsn = PG_GETARG_LSN(1);
+
+	if (IsFutureLSN(start_lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	if (!till_end_of_wal && end_lsn >= curr_lsn)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/*
+	 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+	 * record flushed or replayed respectively. But let's use the LSN up to
+	 * "end".
+	 */
+	if (till_end_of_wal)
+		end_lsn = curr_lsn - 1;
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	wal_details_cb(fcinfo, start_lsn, end_lsn);
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+				  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+		{
+			GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+							  PG_GET_WAL_RECORDS_INFO_COLS);
+
+			tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+								 values, nulls);
+		}
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get info and data of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double	n_pct,
+			rec_len_pct,
+			fpi_len_pct,
+			tot_len_pct;
+	int	i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogStats *stats, ReturnSetInfo *rsinfo,
+					Datum *values, bool *nulls, uint32 ncols)
+{
+	uint64	total_count = 0;
+	uint64	total_rec_len = 0;
+	uint64	total_fpi_len = 0;
+	uint64	total_len = 0;
+	int	ri;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls, ncols);
+
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+							 values, nulls);
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+			XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogStats stats;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_STATS_COLS];
+	bool	nulls[PG_GET_WAL_STATS_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+			XLogRecStoreStats(&stats, xlogreader);
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, rsinfo, values, nulls,
+						PG_GET_WAL_STATS_COLS);
+
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get stats of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWalStats);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get stats of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWalStats);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index 5909ec180e..d9d6889f67 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -1320,13 +1320,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1447,6 +1440,12 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 511f2f186f..7ade716b40 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -79,6 +79,10 @@ typedef struct xl_invalid_page
 
 static HTAB *invalid_page_tab = NULL;
 
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal);
 
 /* Report a reference to an invalid page */
 static void
@@ -851,6 +855,31 @@ wal_segment_close(XLogReaderState *state)
 int
 read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 					 int reqLen, XLogRecPtr targetRecPtr, char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, true);
+}
+
+/*
+ * Same as read_local_xlog_page except that it doesn't wait for future WAL
+ * to be available.
+ */
+int
+read_local_xlog_page_no_wait(XLogReaderState *state, XLogRecPtr targetPagePtr,
+							 int reqLen, XLogRecPtr targetRecPtr,
+							 char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, false);
+}
+
+/*
+ * Implementation of read_local_xlog_page and its no wait version.
+ */
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal)
 {
 	XLogRecPtr	read_upto,
 				loc;
@@ -906,6 +935,10 @@ read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 			if (loc <= read_upto)
 				break;
 
+			/* If asked, let's not wait for future WAL. */
+			if (!wait_for_wal)
+				break;
+
 			CHECK_FOR_INTERRUPTS();
 			pg_usleep(1000L);
 		}
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 179cd617e4..2072fdeed0 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -26,6 +26,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 09f6464331..3e644372f9 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 0e94833129..8468732d76 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index 4021ae05df..fbd251aed2 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -361,9 +361,7 @@ extern void XLogReaderSetDecodeBuffer(XLogReaderState *state,
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Return values from XLogPageReadCB. */
 typedef enum XLogPageReadResult
diff --git a/src/include/access/xlogutils.h b/src/include/access/xlogutils.h
index 64708949db..22c2299d68 100644
--- a/src/include/access/xlogutils.h
+++ b/src/include/access/xlogutils.h
@@ -92,6 +92,10 @@ extern void FreeFakeRelcacheEntry(Relation fakerel);
 extern int	read_local_xlog_page(XLogReaderState *state,
 								 XLogRecPtr targetPagePtr, int reqLen,
 								 XLogRecPtr targetRecPtr, char *cur_page);
+extern int	read_local_xlog_page_no_wait(XLogReaderState *state,
+										 XLogRecPtr targetPagePtr, int reqLen,
+										 XLogRecPtr targetRecPtr,
+										 char *cur_page);
 extern void wal_segment_open(XLogReaderState *state,
 							 XLogSegNo nextSegNo,
 							 TimeLineID *tli_p);
-- 
2.25.1

v14-0003-pg_walinspect-tests.patchapplication/octet-stream; name=v14-0003-pg_walinspect-tests.patchDownload
From e5c03874351e6e9960f070a6f6feb42157cdc2bf Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 24 Mar 2022 09:25:50 +0000
Subject: [PATCH v14] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 216 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 146 ++++++++++++
 2 files changed, 362 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..7445c7dfde
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,216 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+-- Functions accessible by users with role pg_read_server_files
+GRANT pg_read_server_files TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..3a97ef7153
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,146 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+
+-- Functions accessible by users with role pg_read_server_files
+
+GRANT pg_read_server_files TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v14-0004-pg_walinspect-docs.patchapplication/octet-stream; name=v14-0004-pg_walinspect-docs.patchDownload
From f9a0b5c9884e6239eca29b3ab134578af86acb32 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 24 Mar 2022 09:26:42 +0000
Subject: [PATCH v14] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 234 +++++++++++++++++++++++++++++++++
 3 files changed, 236 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index 1e42ce1a7f..4e7b87a42f 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -131,6 +131,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index fd853af01f..34c19c80f1 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -147,6 +147,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..8ab13c38c4
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,234 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_read_server_files</literal> role. Access may be granted by
+  superusers to others using <command>GRANT</command>.
+ </para>
+    
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record(in_lsn pg_lsn,
+                        start_lsn OUT pg_lsn,
+                        end_lsn OUT pg_lsn,
+                        record_length OUT int4,
+                        record OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. This function emits an error if
+      a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text,
+                             data_length OUT int4,
+                             data OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of a given LSN. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text,
+                              data_length OUT int4,
+                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_length, fpi_length, block_ref, data_length from pg_get_wal_records_info('0/1401790', '0/1401D88');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager | record_length | fpi_length |                     block_ref                      | data_length 
+-----------+-----------+-----------+-----+------------------+---------------+------------+----------------------------------------------------+-------------
+ 0/14018B0 | 0/14018EF | 0/1401788 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/14018F0 | 0/1401A17 | 0/14018B0 | 673 | Heap             |           290 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401A18 | 0/1401A57 | 0/14018F0 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401A58 | 0/1401B47 | 0/1401A18 | 673 | Heap             |           234 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401B48 | 0/1401B87 | 0/1401A58 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401B88 | 0/1401C67 | 0/1401B48 | 673 | Heap             |           222 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401C68 | 0/1401CA7 | 0/1401B88 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401CA8 | 0/1401D87 | 0/1401C68 | 673 | Heap             |           217 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+(8 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info_till_end_of_wal(start_lsn pg_lsn,
+                                              start_lsn OUT pg_lsn,
+                                              end_lsn OUT pg_lsn,
+                                              prev_lsn OUT pg_lsn,
+                                              xid OUT xid,
+                                              resource_manager OUT text,
+                                              record_length OUT int4,
+                                              fpi_length OUT int4,
+                                              description OUT text,
+                                              block_ref OUT text,
+                                              data_length OUT int4,
+                                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_records_info()</function>
+      except that it gets information of all the valid WAL records from 
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       resource_manager OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4,
+                       combined_size OUT int8,
+                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/13E80C0', '0/1600000') where count > 0;
+ resource_manager | count | count_percentage | record_size | record_size_percentage | fpi_size | fpi_size_percentage | combined_size | combined_size_percentage 
+------------------+-------+------------------+-------------+------------------------+----------+---------------------+---------------+--------------------------
+ XLOG             |     6 |      0.032156065 |         600 |          1.3965058e-05 |        0 |                   0 |           600 |            1.3963012e-05
+ Transaction      |    61 |       0.32691997 |       16399 |          0.00038168833 |        0 |                   0 |         16399 |             0.0003816324
+ Storage          |     1 |      0.005359344 |          42 |           9.775541e-07 |        0 |                   0 |            42 |             9.774109e-07
+ Database         |     2 |      0.010718688 |          84 |          1.9551082e-06 |        0 |                   0 |            84 |            1.9548218e-06
+ Standby          |   147 |       0.78782356 |       12726 |           0.0002961989 |        0 |                   0 |         12726 |             0.0002961555
+ Heap2            |   804 |        4.3089128 |      190283 |           0.0044288556 |   374096 |            59.41106 |        564379 |              0.013134051
+ Heap             | 16790 |         89.98338 |     1191835 |            0.027740076 |   157528 |            25.01739 |       1349363 |              0.031401955
+ Btree            |   848 |        4.5447235 |  4295025395 |               99.96713 |    98050 |           15.571549 |    4295123445 |                 99.95477
+(8 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats_till_end_of_wal(start_lsn pg_lsn,
+                                       resource_manager OUT text,
+                                       count OUT int8,
+                                       count_percentage OUT float4,
+                                       record_length OUT int8,
+                                       record_length_percentage OUT float4,
+                                       fpi_length OUT int8,
+                                       fpi_length_percentage OUT float4,
+                                       combined_size OUT int8,
+                                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_stats()</function> except
+      that it gets stats of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#78Andres Freund
andres@anarazel.de
In reply to: Bharath Rupireddy (#77)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Hi,

On 2022-03-24 15:02:29 +0530, Bharath Rupireddy wrote:

On Thu, Mar 24, 2022 at 10:22 AM Kyotaro Horiguchi

This doesn't seem to be a part of xlogreader. Couldn't we add a new
module "xlogstats"? XLogRecGetBlockRefInfo also doesn't seem to me as
a part of xlogreader, the xlogstats looks like a better place.

I'm not sure if it's worth adding new files xlogstats.h/.c just for 2
structures, 1 macro, and 2 functions with no plan to add new stats
structures or functions. Since xlogreader is the one that reads the
WAL, and is being included by both backend and other modules (tools
and extensions) IMO it's the right place. However, I can specify in
xlogreader that if at all the new stats related structures or
functions are going to be added, it's good to move them into a new
header and .c file.

I don't like that location for XLogRecGetBlockRefInfo(). How about putting it
in xlogdesc.c - that kind of fits?

And what do you think about creating src/backend/access/rmgrdesc/stats.c for
XLogRecStoreStats()? It's not a perfect location, but not too bad either.

XLogRecGetLen() would be ok in xlogreader, but stats.c also would work?

Greetings,

Andres Freund

#79Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Andres Freund (#78)
4 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Mar 25, 2022 at 12:18 AM Andres Freund <andres@anarazel.de> wrote:

Hi,

On 2022-03-24 15:02:29 +0530, Bharath Rupireddy wrote:

On Thu, Mar 24, 2022 at 10:22 AM Kyotaro Horiguchi

This doesn't seem to be a part of xlogreader. Couldn't we add a new
module "xlogstats"? XLogRecGetBlockRefInfo also doesn't seem to me as
a part of xlogreader, the xlogstats looks like a better place.

I'm not sure if it's worth adding new files xlogstats.h/.c just for 2
structures, 1 macro, and 2 functions with no plan to add new stats
structures or functions. Since xlogreader is the one that reads the
WAL, and is being included by both backend and other modules (tools
and extensions) IMO it's the right place. However, I can specify in
xlogreader that if at all the new stats related structures or
functions are going to be added, it's good to move them into a new
header and .c file.

I don't like that location for XLogRecGetBlockRefInfo(). How about putting it
in xlogdesc.c - that kind of fits?

Done.

And what do you think about creating src/backend/access/rmgrdesc/stats.c for
XLogRecStoreStats()? It's not a perfect location, but not too bad either.

XLogRecGetLen() would be ok in xlogreader, but stats.c also would work?

I've added a new xlogstats.c/.h (as suggested by Kyotaro-san as well)
file under src/backend/access/transam/. I don't think the new file
fits well under rmgrdesc.

Attaching v15 patch-set, please have a look at it.

Regards,
Bharath Rupireddy.

Attachments:

v15-0001-Refactor-pg_waldump-code.patchapplication/x-patch; name=v15-0001-Refactor-pg_waldump-code.patchDownload
From e5a2e473abd98a850406c7991cf1601440be2c97 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 25 Mar 2022 05:57:09 +0000
Subject: [PATCH v15] Refactor pg_waldump code

This patch puts some generic chunks of pg_waldump's code into
separate reusable functions in xlogdesc.c and xlogstats.c, a new
file along xlogstats.h introduced for placing WAL stats and
structures.

This way, other modules can reuse these common functions.
---
 src/backend/access/rmgrdesc/xlogdesc.c | 125 ++++++++++++++++
 src/backend/access/transam/Makefile    |   1 +
 src/backend/access/transam/xlogstats.c |  93 ++++++++++++
 src/bin/pg_waldump/.gitignore          |   1 +
 src/bin/pg_waldump/Makefile            |   8 +-
 src/bin/pg_waldump/pg_waldump.c        | 198 ++-----------------------
 src/include/access/xlog_internal.h     |   5 +
 src/include/access/xlogstats.h         |  40 +++++
 8 files changed, 282 insertions(+), 189 deletions(-)
 create mode 100644 src/backend/access/transam/xlogstats.c
 create mode 100644 src/include/access/xlogstats.h

diff --git a/src/backend/access/rmgrdesc/xlogdesc.c b/src/backend/access/rmgrdesc/xlogdesc.c
index e7452af679..429e5dcd5b 100644
--- a/src/backend/access/rmgrdesc/xlogdesc.c
+++ b/src/backend/access/rmgrdesc/xlogdesc.c
@@ -200,3 +200,128 @@ xlog_identify(uint8 info)
 
 	return id;
 }
+
+/*
+ * Returns a string giving information about all the blocks in an
+ * XLogRecord.
+ */
+void
+XLogRecGetBlockRefInfo(XLogReaderState *record, char *delimiter,
+					   uint32 *fpi_len, bool detailed_format,
+					   StringInfo buf)
+{
+	int	block_id;
+
+	Assert(record != NULL);
+
+	if (detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		RelFileNode rnode = {InvalidOid, InvalidOid, InvalidOid};
+		ForkNumber	forknum = InvalidForkNumber;
+		BlockNumber blk = InvalidBlockNumber;
+
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (detailed_format)
+		{
+			/* Get block references in detailed format. */
+
+			appendStringInfo(buf,
+							 "\tblkref #%d: rel %u/%u/%u fork %s blk %u",
+							 block_id,
+							 rnode.spcNode, rnode.dbNode, rnode.relNode,
+							 forkNames[forknum],
+							 blk);
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
+
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (BKPIMAGE_COMPRESSED(bimg_info))
+				{
+					const char *method;
+
+					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+						method = "pglz";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+						method = "lz4";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
+						method = "zstd";
+					else
+						method = "unknown";
+
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u, "
+									 "compression saved: %u, method: %s",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length,
+									 BLCKSZ -
+									 XLogRecGetBlock(record, block_id)->hole_length -
+									 XLogRecGetBlock(record, block_id)->bimg_len,
+									 method);
+				}
+				else
+				{
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length);
+				}
+			}
+		}
+		else
+		{
+			/* Get block references in short format. */
+
+			if (forknum != MAIN_FORKNUM)
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u fork %s blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 forkNames[forknum],
+								 blk);
+			}
+			else
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 blk);
+			}
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (XLogRecBlockImageApply(record, block_id))
+					appendStringInfo(buf, " FPW");
+				else
+					appendStringInfo(buf, " FPW for WAL verification");
+			}
+		}
+
+		if (detailed_format && delimiter != NULL)
+			appendStringInfoChar(buf, '\n');
+	}
+
+	if (!detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+}
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 79314c69ab..071f3dbe0f 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -33,6 +33,7 @@ OBJS = \
 	xloginsert.o \
 	xlogreader.o \
 	xlogrecovery.o \
+	xlogstats.o \
 	xlogutils.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/transam/xlogstats.c b/src/backend/access/transam/xlogstats.c
new file mode 100644
index 0000000000..aff3069ecb
--- /dev/null
+++ b/src/backend/access/transam/xlogstats.c
@@ -0,0 +1,93 @@
+/*-------------------------------------------------------------------------
+ *
+ * xlogstats.c
+ *		Functions for WAL Statitstics
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		src/backend/access/transam/xlogstats.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlogreader.h"
+#include "access/xlogstats.h"
+
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+void
+XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+			  uint32 *fpi_len)
+{
+	int	block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+void
+XLogRecStoreStats(XLogStats *stats, XLogReaderState *record)
+{
+	RmgrId	rmid;
+	uint8	recid;
+	uint32	rec_len;
+	uint32	fpi_len;
+
+	Assert(stats != NULL && record != NULL);
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	XLogRecGetLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics */
+
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
diff --git a/src/bin/pg_waldump/.gitignore b/src/bin/pg_waldump/.gitignore
index 3be00a8b61..dabb6e34b6 100644
--- a/src/bin/pg_waldump/.gitignore
+++ b/src/bin/pg_waldump/.gitignore
@@ -23,6 +23,7 @@
 /xactdesc.c
 /xlogdesc.c
 /xlogreader.c
+/xlogstat.c
 
 # Generated by test suite
 /tmp_check/
diff --git a/src/bin/pg_waldump/Makefile b/src/bin/pg_waldump/Makefile
index 9f333d0c8a..d6459e17c7 100644
--- a/src/bin/pg_waldump/Makefile
+++ b/src/bin/pg_waldump/Makefile
@@ -13,7 +13,8 @@ OBJS = \
 	compat.o \
 	pg_waldump.o \
 	rmgrdesc.o \
-	xlogreader.o
+	xlogreader.o \
+	xlogstats.o
 
 override CPPFLAGS := -DFRONTEND $(CPPFLAGS)
 
@@ -29,6 +30,9 @@ pg_waldump: $(OBJS) | submake-libpgport
 xlogreader.c: % : $(top_srcdir)/src/backend/access/transam/%
 	rm -f $@ && $(LN_S) $< .
 
+xlogstats.c: % : $(top_srcdir)/src/backend/access/transam/%
+	rm -f $@ && $(LN_S) $< .
+
 $(RMGRDESCSOURCES): % : $(top_srcdir)/src/backend/access/rmgrdesc/%
 	rm -f $@ && $(LN_S) $< .
 
@@ -42,7 +46,7 @@ uninstall:
 	rm -f '$(DESTDIR)$(bindir)/pg_waldump$(X)'
 
 clean distclean maintainer-clean:
-	rm -f pg_waldump$(X) $(OBJS) $(RMGRDESCSOURCES) xlogreader.c
+	rm -f pg_waldump$(X) $(OBJS) $(RMGRDESCSOURCES) xlogreader.c xlogstats.c
 	rm -rf tmp_check
 
 check:
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 9ffe9e55bd..f314d33ebf 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -21,6 +21,7 @@
 #include "access/xlog_internal.h"
 #include "access/xlogreader.h"
 #include "access/xlogrecord.h"
+#include "access/xlogstats.h"
 #include "common/fe_memutils.h"
 #include "common/logging.h"
 #include "getopt_long.h"
@@ -66,23 +67,6 @@ typedef struct XLogDumpConfig
 	bool		filter_by_fpw;
 } XLogDumpConfig;
 
-typedef struct Stats
-{
-	uint64		count;
-	uint64		rec_len;
-	uint64		fpi_len;
-} Stats;
-
-#define MAX_XLINFO_TYPES 16
-
-typedef struct XLogDumpStats
-{
-	uint64		count;
-	XLogRecPtr	startptr;
-	XLogRecPtr	endptr;
-	Stats		rmgr_stats[RM_NEXT_ID];
-	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
-} XLogDumpStats;
 
 #define fatal_error(...) do { pg_log_fatal(__VA_ARGS__); exit(EXIT_FAILURE); } while(0)
 
@@ -453,81 +437,6 @@ XLogRecordHasFPW(XLogReaderState *record)
 	return false;
 }
 
-/*
- * Calculate the size of a record, split into !FPI and FPI parts.
- */
-static void
-XLogDumpRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
-{
-	int			block_id;
-
-	/*
-	 * Calculate the amount of FPI data in the record.
-	 *
-	 * XXX: We peek into xlogreader's private decoded backup blocks for the
-	 * bimg_len indicating the length of FPI data.
-	 */
-	*fpi_len = 0;
-	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-	{
-		if (XLogRecHasBlockImage(record, block_id))
-			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
-	}
-
-	/*
-	 * Calculate the length of the record as the total length - the length of
-	 * all the block images.
-	 */
-	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
-}
-
-/*
- * Store per-rmgr and per-record statistics for a given record.
- */
-static void
-XLogDumpCountRecord(XLogDumpConfig *config, XLogDumpStats *stats,
-					XLogReaderState *record)
-{
-	RmgrId		rmid;
-	uint8		recid;
-	uint32		rec_len;
-	uint32		fpi_len;
-
-	stats->count++;
-
-	rmid = XLogRecGetRmid(record);
-
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
-
-	/* Update per-rmgr statistics */
-
-	stats->rmgr_stats[rmid].count++;
-	stats->rmgr_stats[rmid].rec_len += rec_len;
-	stats->rmgr_stats[rmid].fpi_len += fpi_len;
-
-	/*
-	 * Update per-record statistics, where the record is identified by a
-	 * combination of the RmgrId and the four bits of the xl_info field that
-	 * are the rmgr's domain (resulting in sixteen possible entries per
-	 * RmgrId).
-	 */
-
-	recid = XLogRecGetInfo(record) >> 4;
-
-	/*
-	 * XACT records need to be handled differently. Those records use the
-	 * first bit of those four bits for an optional flag variable and the
-	 * following three bits for the opcode. We filter opcode out of xl_info
-	 * and use it as the identifier of the record.
-	 */
-	if (rmid == RM_XACT_ID)
-		recid &= 0x07;
-
-	stats->record_stats[rmid][recid].count++;
-	stats->record_stats[rmid][recid].rec_len += rec_len;
-	stats->record_stats[rmid][recid].fpi_len += fpi_len;
-}
-
 /*
  * Print a record to stdout
  */
@@ -538,15 +447,12 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	const RmgrDescData *desc = &RmgrDescTable[XLogRecGetRmid(record)];
 	uint32		rec_len;
 	uint32		fpi_len;
-	RelFileNode rnode;
-	ForkNumber	forknum;
-	BlockNumber blk;
-	int			block_id;
 	uint8		info = XLogRecGetInfo(record);
 	XLogRecPtr	xl_prev = XLogRecGetPrev(record);
 	StringInfoData s;
+	char	delim = {'\n'};
 
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
+	XLogRecGetLen(record, &rec_len, &fpi_len);
 
 	printf("rmgr: %-11s len (rec/tot): %6u/%6u, tx: %10u, lsn: %X/%08X, prev %X/%08X, ",
 		   desc->rm_name,
@@ -564,93 +470,11 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	initStringInfo(&s);
 	desc->rm_desc(&s, record);
 	printf("%s", s.data);
-	pfree(s.data);
-
-	if (!config->bkp_details)
-	{
-		/* print block references (short format) */
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			if (forknum != MAIN_FORKNUM)
-				printf(", blkref #%d: rel %u/%u/%u fork %s blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   forkNames[forknum],
-					   blk);
-			else
-				printf(", blkref #%d: rel %u/%u/%u blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				if (XLogRecBlockImageApply(record, block_id))
-					printf(" FPW");
-				else
-					printf(" FPW for WAL verification");
-			}
-		}
-		putchar('\n');
-	}
-	else
-	{
-		/* print block references (detailed format) */
-		putchar('\n');
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			printf("\tblkref #%d: rel %u/%u/%u fork %s blk %u",
-				   block_id,
-				   rnode.spcNode, rnode.dbNode, rnode.relNode,
-				   forkNames[forknum],
-				   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
 
-				if (BKPIMAGE_COMPRESSED(bimg_info))
-				{
-					const char *method;
-
-					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
-						method = "pglz";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
-						method = "lz4";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
-						method = "zstd";
-					else
-						method = "unknown";
-
-					printf(" (FPW%s); hole: offset: %u, length: %u, "
-						   "compression saved: %u, method: %s",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length,
-						   BLCKSZ -
-						   XLogRecGetBlock(record, block_id)->hole_length -
-						   XLogRecGetBlock(record, block_id)->bimg_len,
-						   method);
-				}
-				else
-				{
-					printf(" (FPW%s); hole: offset: %u, length: %u",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length);
-				}
-			}
-			putchar('\n');
-		}
-	}
+	resetStringInfo(&s);
+	XLogRecGetBlockRefInfo(record, &delim, NULL, config->bkp_details, &s);
+	printf("%s", s.data);
+	pfree(s.data);
 }
 
 /*
@@ -698,7 +522,7 @@ XLogDumpStatsRow(const char *name,
  * Display summary statistics about the records seen so far.
  */
 static void
-XLogDumpDisplayStats(XLogDumpConfig *config, XLogDumpStats *stats)
+XLogDumpDisplayStats(XLogDumpConfig *config, XLogStats *stats)
 {
 	int			ri,
 				rj;
@@ -859,7 +683,7 @@ main(int argc, char **argv)
 	XLogReaderState *xlogreader_state;
 	XLogDumpPrivate private;
 	XLogDumpConfig config;
-	XLogDumpStats stats;
+	XLogStats stats;
 	XLogRecord *record;
 	XLogRecPtr	first_record;
 	char	   *waldir = NULL;
@@ -913,7 +737,7 @@ main(int argc, char **argv)
 
 	memset(&private, 0, sizeof(XLogDumpPrivate));
 	memset(&config, 0, sizeof(XLogDumpConfig));
-	memset(&stats, 0, sizeof(XLogDumpStats));
+	memset(&stats, 0, sizeof(XLogStats));
 
 	private.timeline = 1;
 	private.startptr = InvalidXLogRecPtr;
@@ -1289,7 +1113,7 @@ main(int argc, char **argv)
 		{
 			if (config.stats == true)
 			{
-				XLogDumpCountRecord(&config, &stats, xlogreader_state);
+				XLogRecStoreStats(&stats, xlogreader_state);
 				stats.endptr = xlogreader_state->EndRecPtr;
 			}
 			else
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 0e94833129..d7c35c37c4 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -329,6 +329,11 @@ extern XLogRecPtr RequestXLogSwitch(bool mark_unimportant);
 
 extern void GetOldestRestartPoint(XLogRecPtr *oldrecptr, TimeLineID *oldtli);
 
+extern void XLogRecGetBlockRefInfo(XLogReaderState *record,
+								   char *delimiter, uint32 *fpi_len,
+								   bool detailed_format,
+								   StringInfo blk_ref);
+
 /*
  * Exported for the functions in timeline.c and xlogarchive.c.  Only valid
  * in the startup process.
diff --git a/src/include/access/xlogstats.h b/src/include/access/xlogstats.h
new file mode 100644
index 0000000000..36d833f82b
--- /dev/null
+++ b/src/include/access/xlogstats.h
@@ -0,0 +1,40 @@
+/*-------------------------------------------------------------------------
+ *
+ * xlogstats.h
+ *		Definitions for WAL Statitstics
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		src/include/access/xlogstats.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef XLOGSTATS_H
+#define XLOGSTATS_H
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64	count;
+	uint64	rec_len;
+	uint64	fpi_len;
+} XLogRecStats;
+
+typedef struct XLogStats
+{
+	uint64	count;
+#ifdef FRONTEND
+	XLogRecPtr	startptr;
+	XLogRecPtr	endptr;
+#endif
+	XLogRecStats    rmgr_stats[RM_NEXT_ID];
+	XLogRecStats    record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+} XLogStats;
+
+extern void XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+						  uint32 *fpi_len);
+extern void XLogRecStoreStats(XLogStats *stats, XLogReaderState *record);
+
+#endif							/* XLOGSTATS_H */
-- 
2.25.1

v15-0002-pg_walinspect.patchapplication/x-patch; name=v15-0002-pg_walinspect.patchDownload
From 61873a762167c2a4a431d0803b163794883f3b21 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 25 Mar 2022 06:10:26 +0000
Subject: [PATCH v15] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 132 ++++
 contrib/pg_walinspect/pg_walinspect.c        | 653 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  13 +-
 src/backend/access/transam/xlogutils.c       |  33 +
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/access/xlogutils.h               |   4 +
 13 files changed, 871 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 332b486ecc..bbf220407b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -41,6 +41,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..9b192dd2c1
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,132 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_wal_record()
+--
+CREATE FUNCTION pg_get_wal_record(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT record_length int4,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn) TO pg_read_server_files;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..0aa4c0aeed
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,653 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogstats.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(pg_get_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats_till_end_of_wal);
+
+typedef void (*GetWALDetailsCB) (FunctionCallInfo fcinfo,
+								 XLogRecPtr start_lsn,
+								 XLogRecPtr end_lsn);
+
+static bool IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static Datum GetWALRecordInternal(FunctionCallInfo fcinfo, Datum *values,
+								  bool *nulls, uint32 ncols, bool get_info);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+							  GetWALDetailsCB wal_details_cb);
+static void GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+							  XLogRecPtr end_lsn);
+static void GetXLogSummaryStats(XLogStats * stats, ReturnSetInfo *rsinfo,
+								Datum *values, bool *nulls, uint32 ncols);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						XLogRecPtr end_lsn);
+
+/*
+ * Determinte if the given LSN is in future and return the LSN up to which the
+ * server has WAL.
+ */
+static bool
+IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn)
+{
+	/*
+	 * We determine the current LSN of the server similar to how page_read
+	 * callback read_local_xlog_page_no_wait does.
+	 */
+	if (!RecoveryInProgress())
+		*curr_lsn = GetFlushRecPtr(NULL);
+	else
+		*curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(*curr_lsn));
+
+	if (lsn >= *curr_lsn)
+		return true;
+
+	return false;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page_no_wait,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	initStringInfo(&rec_desc);
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	desc->rm_desc(&rec_desc, record);
+
+	/* Block references. */
+	initStringInfo(&rec_blk_ref);
+	XLogRecGetBlockRefInfo(record, NULL, &fpi_len, true, &rec_blk_ref);
+
+	main_data_len = XLogRecGetDataLen(record);
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = PointerGetDatum(data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record data or info.
+ */
+Datum
+GetWALRecordInternal(FunctionCallInfo fcinfo, Datum *values, bool *nulls,
+					 uint32 ncols, bool get_info)
+{
+	XLogRecPtr	lsn;
+	XLogRecPtr	curr_lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	HeapTuple	tuple;
+	Datum	result;
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (IsFutureLSN(lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, ncols);
+	MemSet(nulls, 0, ncols);
+
+	if (get_info)
+	{
+		GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+						  ncols);
+	}
+	else
+	{
+		bytea	*rec;
+		uint32	rec_len;
+		char	*rec_data;
+		int	i = 0;
+
+		rec_len = XLogRecGetTotalLen(xlogreader);
+
+		Assert(rec_len > 0);
+
+		rec = (bytea *) palloc(rec_len + VARHDRSZ);
+		SET_VARSIZE(rec, rec_len + VARHDRSZ);
+		rec_data = VARDATA(rec);
+
+		memcpy(rec_data, record, rec_len);
+
+		values[i++] = LSNGetDatum(first_record);
+		values[i++] = LSNGetDatum(xlogreader->EndRecPtr - 1);
+		values[i++] = LSNGetDatum(XLogRecGetPrev(xlogreader));
+		values[i++] = UInt32GetDatum(rec_len);
+		values[i++] = PointerGetDatum(rec);
+
+		Assert(i == ncols);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	return result;
+}
+
+/*
+ * Get WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_COLS 5
+	Datum	result;
+	Datum	values[PG_GET_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_COLS];
+
+	result = GetWALRecordInternal(fcinfo, values, nulls,
+								  PG_GET_WAL_RECORD_COLS,
+								  false);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_COLS
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	Datum	result;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	result = GetWALRecordInternal(fcinfo, values, nulls,
+								  PG_GET_WAL_RECORD_INFO_COLS,
+								  true);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get WAL details such as record info, stats using the passed in callback.
+ */
+static void
+GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+				  GetWALDetailsCB wal_details_cb)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecPtr	curr_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* If not till end of wal, end_lsn would have been specified. */
+	if (!till_end_of_wal)
+		end_lsn = PG_GETARG_LSN(1);
+
+	if (IsFutureLSN(start_lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	if (!till_end_of_wal && end_lsn >= curr_lsn)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/*
+	 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+	 * record flushed or replayed respectively. But let's use the LSN up to
+	 * "end".
+	 */
+	if (till_end_of_wal)
+		end_lsn = curr_lsn - 1;
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	wal_details_cb(fcinfo, start_lsn, end_lsn);
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+				  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+		{
+			GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+							  PG_GET_WAL_RECORDS_INFO_COLS);
+
+			tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+								 values, nulls);
+		}
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get info and data of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double	n_pct,
+			rec_len_pct,
+			fpi_len_pct,
+			tot_len_pct;
+	int	i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogStats *stats, ReturnSetInfo *rsinfo,
+					Datum *values, bool *nulls, uint32 ncols)
+{
+	uint64	total_count = 0;
+	uint64	total_rec_len = 0;
+	uint64	total_fpi_len = 0;
+	uint64	total_len = 0;
+	int	ri;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls, ncols);
+
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+							 values, nulls);
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+			XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogStats stats;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_STATS_COLS];
+	bool	nulls[PG_GET_WAL_STATS_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+			XLogRecStoreStats(&stats, xlogreader);
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, rsinfo, values, nulls,
+						PG_GET_WAL_STATS_COLS);
+
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get stats of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWalStats);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get stats of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWalStats);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index e437c42992..585c94c488 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -1320,13 +1320,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1447,6 +1440,12 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 511f2f186f..7ade716b40 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -79,6 +79,10 @@ typedef struct xl_invalid_page
 
 static HTAB *invalid_page_tab = NULL;
 
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal);
 
 /* Report a reference to an invalid page */
 static void
@@ -851,6 +855,31 @@ wal_segment_close(XLogReaderState *state)
 int
 read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 					 int reqLen, XLogRecPtr targetRecPtr, char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, true);
+}
+
+/*
+ * Same as read_local_xlog_page except that it doesn't wait for future WAL
+ * to be available.
+ */
+int
+read_local_xlog_page_no_wait(XLogReaderState *state, XLogRecPtr targetPagePtr,
+							 int reqLen, XLogRecPtr targetRecPtr,
+							 char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, false);
+}
+
+/*
+ * Implementation of read_local_xlog_page and its no wait version.
+ */
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal)
 {
 	XLogRecPtr	read_upto,
 				loc;
@@ -906,6 +935,10 @@ read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 			if (loc <= read_upto)
 				break;
 
+			/* If asked, let's not wait for future WAL. */
+			if (!wait_for_wal)
+				break;
+
 			CHECK_FOR_INTERRUPTS();
 			pg_usleep(1000L);
 		}
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index f314d33ebf..cad5b3594a 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -27,6 +27,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 09f6464331..3e644372f9 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index d7c35c37c4..2985c75361 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index f4388cc9be..b4c7d93787 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -340,9 +340,7 @@ extern void XLogReaderSetDecodeBuffer(XLogReaderState *state,
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Return values from XLogPageReadCB. */
 typedef enum XLogPageReadResult
diff --git a/src/include/access/xlogutils.h b/src/include/access/xlogutils.h
index 64708949db..22c2299d68 100644
--- a/src/include/access/xlogutils.h
+++ b/src/include/access/xlogutils.h
@@ -92,6 +92,10 @@ extern void FreeFakeRelcacheEntry(Relation fakerel);
 extern int	read_local_xlog_page(XLogReaderState *state,
 								 XLogRecPtr targetPagePtr, int reqLen,
 								 XLogRecPtr targetRecPtr, char *cur_page);
+extern int	read_local_xlog_page_no_wait(XLogReaderState *state,
+										 XLogRecPtr targetPagePtr, int reqLen,
+										 XLogRecPtr targetRecPtr,
+										 char *cur_page);
 extern void wal_segment_open(XLogReaderState *state,
 							 XLogSegNo nextSegNo,
 							 TimeLineID *tli_p);
-- 
2.25.1

v15-0003-pg_walinspect-tests.patchapplication/x-patch; name=v15-0003-pg_walinspect-tests.patchDownload
From 9a9358572fa7877077f9962d5894a8cad63b5f53 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 25 Mar 2022 06:11:38 +0000
Subject: [PATCH v15] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 216 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 146 ++++++++++++
 2 files changed, 362 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..7445c7dfde
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,216 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+-- Functions accessible by users with role pg_read_server_files
+GRANT pg_read_server_files TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..3a97ef7153
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,146 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+
+-- Functions accessible by users with role pg_read_server_files
+
+GRANT pg_read_server_files TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v15-0004-pg_walinspect-docs.patchapplication/x-patch; name=v15-0004-pg_walinspect-docs.patchDownload
From 0d13fbd5f265f41a8f53de8b2c35a2b60bd53779 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Fri, 25 Mar 2022 06:12:45 +0000
Subject: [PATCH v15] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 234 +++++++++++++++++++++++++++++++++
 3 files changed, 236 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index 1e42ce1a7f..4e7b87a42f 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -131,6 +131,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index fd853af01f..34c19c80f1 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -147,6 +147,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..8ab13c38c4
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,234 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_read_server_files</literal> role. Access may be granted by
+  superusers to others using <command>GRANT</command>.
+ </para>
+    
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record(in_lsn pg_lsn,
+                        start_lsn OUT pg_lsn,
+                        end_lsn OUT pg_lsn,
+                        record_length OUT int4,
+                        record OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. This function emits an error if
+      a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text,
+                             data_length OUT int4,
+                             data OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of a given LSN. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text,
+                              data_length OUT int4,
+                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_length, fpi_length, block_ref, data_length from pg_get_wal_records_info('0/1401790', '0/1401D88');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager | record_length | fpi_length |                     block_ref                      | data_length 
+-----------+-----------+-----------+-----+------------------+---------------+------------+----------------------------------------------------+-------------
+ 0/14018B0 | 0/14018EF | 0/1401788 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/14018F0 | 0/1401A17 | 0/14018B0 | 673 | Heap             |           290 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401A18 | 0/1401A57 | 0/14018F0 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401A58 | 0/1401B47 | 0/1401A18 | 673 | Heap             |           234 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401B48 | 0/1401B87 | 0/1401A58 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401B88 | 0/1401C67 | 0/1401B48 | 673 | Heap             |           222 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401C68 | 0/1401CA7 | 0/1401B88 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401CA8 | 0/1401D87 | 0/1401C68 | 673 | Heap             |           217 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+(8 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info_till_end_of_wal(start_lsn pg_lsn,
+                                              start_lsn OUT pg_lsn,
+                                              end_lsn OUT pg_lsn,
+                                              prev_lsn OUT pg_lsn,
+                                              xid OUT xid,
+                                              resource_manager OUT text,
+                                              record_length OUT int4,
+                                              fpi_length OUT int4,
+                                              description OUT text,
+                                              block_ref OUT text,
+                                              data_length OUT int4,
+                                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_records_info()</function>
+      except that it gets information of all the valid WAL records from 
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       resource_manager OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4,
+                       combined_size OUT int8,
+                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/13E80C0', '0/1600000') where count > 0;
+ resource_manager | count | count_percentage | record_size | record_size_percentage | fpi_size | fpi_size_percentage | combined_size | combined_size_percentage 
+------------------+-------+------------------+-------------+------------------------+----------+---------------------+---------------+--------------------------
+ XLOG             |     6 |      0.032156065 |         600 |          1.3965058e-05 |        0 |                   0 |           600 |            1.3963012e-05
+ Transaction      |    61 |       0.32691997 |       16399 |          0.00038168833 |        0 |                   0 |         16399 |             0.0003816324
+ Storage          |     1 |      0.005359344 |          42 |           9.775541e-07 |        0 |                   0 |            42 |             9.774109e-07
+ Database         |     2 |      0.010718688 |          84 |          1.9551082e-06 |        0 |                   0 |            84 |            1.9548218e-06
+ Standby          |   147 |       0.78782356 |       12726 |           0.0002961989 |        0 |                   0 |         12726 |             0.0002961555
+ Heap2            |   804 |        4.3089128 |      190283 |           0.0044288556 |   374096 |            59.41106 |        564379 |              0.013134051
+ Heap             | 16790 |         89.98338 |     1191835 |            0.027740076 |   157528 |            25.01739 |       1349363 |              0.031401955
+ Btree            |   848 |        4.5447235 |  4295025395 |               99.96713 |    98050 |           15.571549 |    4295123445 |                 99.95477
+(8 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats_till_end_of_wal(start_lsn pg_lsn,
+                                       resource_manager OUT text,
+                                       count OUT int8,
+                                       count_percentage OUT float4,
+                                       record_length OUT int8,
+                                       record_length_percentage OUT float4,
+                                       fpi_length OUT int8,
+                                       fpi_length_percentage OUT float4,
+                                       combined_size OUT int8,
+                                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_stats()</function> except
+      that it gets stats of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#80RKN Sai Krishna
rknsaiforpostgres@gmail.com
In reply to: Bharath Rupireddy (#79)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Hi Bharath,

First look at the patch, bear with me if any of the following comments are
repeated.
1. With pg_get_wal_record(lsn), say a WAL record start, end lsn range
contains the specified LSN, wouldn't it be more meaningful to show the
corresponding WAL record.
For example, upon providing '0/17335E7' as input, and I see get the WAL
record ('0/1733618', '0/173409F') as output and not the one with start and
end lsn as ('0/17335E0', '0/1733617').

With pg_walfile_name(lsn), we can find the WAL segment file name that
contains the specified LSN.

2. I see the following output for pg_get_wal_record. Need to have a look at
the spaces I suppose.
rkn=# select * from pg_get_wal_record('0/4041728');
start_lsn | end_lsn | prev_lsn | record_length |

record

-----------+-----------+-----------+---------------+---------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------------------------------------------------------------------------
0/4041728 | 0/40421AF | 0/40416F0 | 2670 |
\x6e0a0000d2020000f016040400000000000a0000fef802b400007f7f00000000408fe738a25500000300000000000000010000007f06000000
4000003b0a00000000000012000000100101007f7f7f7f0885e738a25500003000c815380a03000885e738a255000000007f7f7f7f7f7f0000000078674301296000003000f8150020042000000000709e1203909
dba01c89c8601609cd00030986008f8956804d202000000000000000000000000120006001f00030820ffff5f04000000000001400000010000000000000004000000000080bf0200030000000000000000006100
0000610000000000000000000000000000000000000000000000000000000000000000000000330100000000000000bc02000001000000010000000000803f00000000000000b0060000010000000000000017000

3. Should these functions be running in standby mode too? We do not allow
WAL control functions to be executed during recovery right?

4. If the wal segment corresponding to the start lsn is removed, but there
are WAL records which could be read in the specified input lsn range, would
it be better to output the existing WAL records displaying a message that
it is a partial list of WAL records and the WAL files corresponding to the
rest are already removed, rather than erroring out saying "requested WAL
segment has already been removed"?

5. Following are very minor comments in the code

- Correct the function description by removing "return the LSN up to
which the server has WAL" for IsFutureLSN
- In GetXLogRecordInfo, good to have pfree in place for rec_desc,
rec_blk_ref, data
- In GetXLogRecordInfo, can avoid calling XLogRecGetInfo(record)
multiple times by capturing in a variable
- In GetWALDetailsGuts, setting end_lsn could be done in single if else
and similarly we can club the if statements verifying if the start lsn is a
future lsn.

Thanks,
RKN

#81Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: RKN Sai Krishna (#80)
4 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Fri, Mar 25, 2022 at 8:37 PM RKN Sai Krishna
<rknsaiforpostgres@gmail.com> wrote:

Hi Bharath,

First look at the patch, bear with me if any of the following comments are repeated.

Thanks RKN, for playing around with the patches.

1. With pg_get_wal_record(lsn), say a WAL record start, end lsn range contains the specified LSN, wouldn't it be more meaningful to show the corresponding WAL record.

In general, all the functions will first look for a first valid WAL
record from the given input lsn/start lsn(XLogFindNextRecord) and then
give info of all the valid records including the first valid WAL
record until either the given end lsn or till end of WAL depending on
the function used.

For example, upon providing '0/17335E7' as input, and I see get the WAL record ('0/1733618', '0/173409F') as output and not the one with start and end lsn as ('0/17335E0', '0/1733617').

If '0/17335E7' is an LSN containing a valid WAL record,
pg_get_wal_record gives the info of that, otherwise if there's any
next valid WAL record, it finds and gives that info. '0/17335E0' is
before '0/17335E7' the input lsn, so it doesn't show that record, but
the next valid record.

All the pg_walinspect functions don't look for the nearest valid WAL
record (could be previous to input lsn or next to input lsn), but they
look for the next valid WAL record. This is because the xlogreader
infra now has no API for backward iteration from a given LSN ( it has
XLogFindNextRecord and XLogReadRecord which scans the WAL in forward
direction). But, it's a good idea to have XLogFindPreviousRecord and
XLogReadPreviousRecord versions (as we have links for previous WAL
record in each WAL record) but that's a separate discussion.

With pg_walfile_name(lsn), we can find the WAL segment file name that contains the specified LSN.

Yes.

2. I see the following output for pg_get_wal_record. Need to have a look at the spaces I suppose.

I believe this is something psql does for larger column outputs for
pretty-display. When used in a non-psql client, the column values are
returned properly. Nothing to do with the pg_walinspect patches here.

3. Should these functions be running in standby mode too? We do not allow WAL control functions to be executed during recovery right?

There are functions that can be executable during recovery
pg_last_wal_receive_lsn, pg_last_wal_replay_lsn. The pg_walinspect
functions are useful even in recovery and I don't see a strong reason
to not support them. Hence, I'm right now supporting them.

4. If the wal segment corresponding to the start lsn is removed, but there are WAL records which could be read in the specified input lsn range, would it be better to output the existing WAL records displaying a message that it is a partial list of WAL records and the WAL files corresponding to the rest are already removed, rather than erroring out saying "requested WAL segment has already been removed"?

"requested WAL segment %s has already been removed" is a common error
across the xlogreader infra (see wal_segment_open) and I don't want to
invent a new behaviour. And all the segment_open callbacks report an
error when they are not finding the WAL file that they are looking
for.

5. Following are very minor comments in the code

Correct the function description by removing "return the LSN up to which the server has WAL" for IsFutureLSN

That's fine, because it actually returns curr_lsn via the function
param curr_lsn. However, I modified the comment a bit.

In GetXLogRecordInfo, good to have pfree in place for rec_desc, rec_blk_ref, data

No, we are just returning pointer to the string, not deep copying, see
CStringGetTextDatum. All the functions get executed within a
function's memory context and after handing off the results to the
client that gets deleted, deallocating all the memory.

In GetXLogRecordInfo, can avoid calling XLogRecGetInfo(record) multiple times by capturing in a variable

XLogRecGetInfo is not a function, it's a macro, so that's fine.
#define XLogRecGetInfo(decoder) ((decoder)->record->header.xl_info)

In GetWALDetailsGuts, setting end_lsn could be done in single if else and similarly we can club the if statements verifying if the start lsn is a future lsn.

The existing if conditions are:

if (IsFutureLSN(start_lsn, &curr_lsn))
if (!till_end_of_wal && end_lsn >= curr_lsn)
if (till_end_of_wal)
if (start_lsn >= end_lsn)

I clubbed them like this:
if (!till_end_of_wal)
if (IsFutureLSN(start_lsn, &curr_lsn))
if (!till_end_of_wal && end_lsn >= curr_lsn)
else if (till_end_of_wal)

Other if conditions are serving different purposes, so I'm leaving them as-is.

Attaching v16 patch-set, only change in v16-0002-pg_walinspect.patch,
others remain the same.

Regards,
Bharath Rupireddy.

Attachments:

v16-0001-Refactor-pg_waldump-code.patchapplication/octet-stream; name=v16-0001-Refactor-pg_waldump-code.patchDownload
From 173419ee42660c3b186c5149747fb8f099cc4107 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sat, 26 Mar 2022 04:47:13 +0000
Subject: [PATCH v16] Refactor pg_waldump code

This patch puts some generic chunks of pg_waldump's code into
separate reusable functions in xlogdesc.c and xlogstats.c, a new
file along xlogstats.h introduced for placing WAL stats and
structures.

This way, other modules can reuse these common functions.
---
 src/backend/access/rmgrdesc/xlogdesc.c | 125 ++++++++++++++++
 src/backend/access/transam/Makefile    |   1 +
 src/backend/access/transam/xlogstats.c |  93 ++++++++++++
 src/bin/pg_waldump/.gitignore          |   1 +
 src/bin/pg_waldump/Makefile            |   8 +-
 src/bin/pg_waldump/pg_waldump.c        | 198 ++-----------------------
 src/include/access/xlog_internal.h     |   5 +
 src/include/access/xlogstats.h         |  40 +++++
 8 files changed, 282 insertions(+), 189 deletions(-)
 create mode 100644 src/backend/access/transam/xlogstats.c
 create mode 100644 src/include/access/xlogstats.h

diff --git a/src/backend/access/rmgrdesc/xlogdesc.c b/src/backend/access/rmgrdesc/xlogdesc.c
index e7452af679..429e5dcd5b 100644
--- a/src/backend/access/rmgrdesc/xlogdesc.c
+++ b/src/backend/access/rmgrdesc/xlogdesc.c
@@ -200,3 +200,128 @@ xlog_identify(uint8 info)
 
 	return id;
 }
+
+/*
+ * Returns a string giving information about all the blocks in an
+ * XLogRecord.
+ */
+void
+XLogRecGetBlockRefInfo(XLogReaderState *record, char *delimiter,
+					   uint32 *fpi_len, bool detailed_format,
+					   StringInfo buf)
+{
+	int	block_id;
+
+	Assert(record != NULL);
+
+	if (detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		RelFileNode rnode = {InvalidOid, InvalidOid, InvalidOid};
+		ForkNumber	forknum = InvalidForkNumber;
+		BlockNumber blk = InvalidBlockNumber;
+
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (detailed_format)
+		{
+			/* Get block references in detailed format. */
+
+			appendStringInfo(buf,
+							 "\tblkref #%d: rel %u/%u/%u fork %s blk %u",
+							 block_id,
+							 rnode.spcNode, rnode.dbNode, rnode.relNode,
+							 forkNames[forknum],
+							 blk);
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
+
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (BKPIMAGE_COMPRESSED(bimg_info))
+				{
+					const char *method;
+
+					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+						method = "pglz";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+						method = "lz4";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
+						method = "zstd";
+					else
+						method = "unknown";
+
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u, "
+									 "compression saved: %u, method: %s",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length,
+									 BLCKSZ -
+									 XLogRecGetBlock(record, block_id)->hole_length -
+									 XLogRecGetBlock(record, block_id)->bimg_len,
+									 method);
+				}
+				else
+				{
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length);
+				}
+			}
+		}
+		else
+		{
+			/* Get block references in short format. */
+
+			if (forknum != MAIN_FORKNUM)
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u fork %s blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 forkNames[forknum],
+								 blk);
+			}
+			else
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 blk);
+			}
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (XLogRecBlockImageApply(record, block_id))
+					appendStringInfo(buf, " FPW");
+				else
+					appendStringInfo(buf, " FPW for WAL verification");
+			}
+		}
+
+		if (detailed_format && delimiter != NULL)
+			appendStringInfoChar(buf, '\n');
+	}
+
+	if (!detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+}
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 79314c69ab..071f3dbe0f 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -33,6 +33,7 @@ OBJS = \
 	xloginsert.o \
 	xlogreader.o \
 	xlogrecovery.o \
+	xlogstats.o \
 	xlogutils.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/transam/xlogstats.c b/src/backend/access/transam/xlogstats.c
new file mode 100644
index 0000000000..aff3069ecb
--- /dev/null
+++ b/src/backend/access/transam/xlogstats.c
@@ -0,0 +1,93 @@
+/*-------------------------------------------------------------------------
+ *
+ * xlogstats.c
+ *		Functions for WAL Statitstics
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		src/backend/access/transam/xlogstats.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlogreader.h"
+#include "access/xlogstats.h"
+
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+void
+XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+			  uint32 *fpi_len)
+{
+	int	block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+void
+XLogRecStoreStats(XLogStats *stats, XLogReaderState *record)
+{
+	RmgrId	rmid;
+	uint8	recid;
+	uint32	rec_len;
+	uint32	fpi_len;
+
+	Assert(stats != NULL && record != NULL);
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	XLogRecGetLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics */
+
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
diff --git a/src/bin/pg_waldump/.gitignore b/src/bin/pg_waldump/.gitignore
index 3be00a8b61..dabb6e34b6 100644
--- a/src/bin/pg_waldump/.gitignore
+++ b/src/bin/pg_waldump/.gitignore
@@ -23,6 +23,7 @@
 /xactdesc.c
 /xlogdesc.c
 /xlogreader.c
+/xlogstat.c
 
 # Generated by test suite
 /tmp_check/
diff --git a/src/bin/pg_waldump/Makefile b/src/bin/pg_waldump/Makefile
index 9f333d0c8a..d6459e17c7 100644
--- a/src/bin/pg_waldump/Makefile
+++ b/src/bin/pg_waldump/Makefile
@@ -13,7 +13,8 @@ OBJS = \
 	compat.o \
 	pg_waldump.o \
 	rmgrdesc.o \
-	xlogreader.o
+	xlogreader.o \
+	xlogstats.o
 
 override CPPFLAGS := -DFRONTEND $(CPPFLAGS)
 
@@ -29,6 +30,9 @@ pg_waldump: $(OBJS) | submake-libpgport
 xlogreader.c: % : $(top_srcdir)/src/backend/access/transam/%
 	rm -f $@ && $(LN_S) $< .
 
+xlogstats.c: % : $(top_srcdir)/src/backend/access/transam/%
+	rm -f $@ && $(LN_S) $< .
+
 $(RMGRDESCSOURCES): % : $(top_srcdir)/src/backend/access/rmgrdesc/%
 	rm -f $@ && $(LN_S) $< .
 
@@ -42,7 +46,7 @@ uninstall:
 	rm -f '$(DESTDIR)$(bindir)/pg_waldump$(X)'
 
 clean distclean maintainer-clean:
-	rm -f pg_waldump$(X) $(OBJS) $(RMGRDESCSOURCES) xlogreader.c
+	rm -f pg_waldump$(X) $(OBJS) $(RMGRDESCSOURCES) xlogreader.c xlogstats.c
 	rm -rf tmp_check
 
 check:
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 9ffe9e55bd..f314d33ebf 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -21,6 +21,7 @@
 #include "access/xlog_internal.h"
 #include "access/xlogreader.h"
 #include "access/xlogrecord.h"
+#include "access/xlogstats.h"
 #include "common/fe_memutils.h"
 #include "common/logging.h"
 #include "getopt_long.h"
@@ -66,23 +67,6 @@ typedef struct XLogDumpConfig
 	bool		filter_by_fpw;
 } XLogDumpConfig;
 
-typedef struct Stats
-{
-	uint64		count;
-	uint64		rec_len;
-	uint64		fpi_len;
-} Stats;
-
-#define MAX_XLINFO_TYPES 16
-
-typedef struct XLogDumpStats
-{
-	uint64		count;
-	XLogRecPtr	startptr;
-	XLogRecPtr	endptr;
-	Stats		rmgr_stats[RM_NEXT_ID];
-	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
-} XLogDumpStats;
 
 #define fatal_error(...) do { pg_log_fatal(__VA_ARGS__); exit(EXIT_FAILURE); } while(0)
 
@@ -453,81 +437,6 @@ XLogRecordHasFPW(XLogReaderState *record)
 	return false;
 }
 
-/*
- * Calculate the size of a record, split into !FPI and FPI parts.
- */
-static void
-XLogDumpRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
-{
-	int			block_id;
-
-	/*
-	 * Calculate the amount of FPI data in the record.
-	 *
-	 * XXX: We peek into xlogreader's private decoded backup blocks for the
-	 * bimg_len indicating the length of FPI data.
-	 */
-	*fpi_len = 0;
-	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-	{
-		if (XLogRecHasBlockImage(record, block_id))
-			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
-	}
-
-	/*
-	 * Calculate the length of the record as the total length - the length of
-	 * all the block images.
-	 */
-	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
-}
-
-/*
- * Store per-rmgr and per-record statistics for a given record.
- */
-static void
-XLogDumpCountRecord(XLogDumpConfig *config, XLogDumpStats *stats,
-					XLogReaderState *record)
-{
-	RmgrId		rmid;
-	uint8		recid;
-	uint32		rec_len;
-	uint32		fpi_len;
-
-	stats->count++;
-
-	rmid = XLogRecGetRmid(record);
-
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
-
-	/* Update per-rmgr statistics */
-
-	stats->rmgr_stats[rmid].count++;
-	stats->rmgr_stats[rmid].rec_len += rec_len;
-	stats->rmgr_stats[rmid].fpi_len += fpi_len;
-
-	/*
-	 * Update per-record statistics, where the record is identified by a
-	 * combination of the RmgrId and the four bits of the xl_info field that
-	 * are the rmgr's domain (resulting in sixteen possible entries per
-	 * RmgrId).
-	 */
-
-	recid = XLogRecGetInfo(record) >> 4;
-
-	/*
-	 * XACT records need to be handled differently. Those records use the
-	 * first bit of those four bits for an optional flag variable and the
-	 * following three bits for the opcode. We filter opcode out of xl_info
-	 * and use it as the identifier of the record.
-	 */
-	if (rmid == RM_XACT_ID)
-		recid &= 0x07;
-
-	stats->record_stats[rmid][recid].count++;
-	stats->record_stats[rmid][recid].rec_len += rec_len;
-	stats->record_stats[rmid][recid].fpi_len += fpi_len;
-}
-
 /*
  * Print a record to stdout
  */
@@ -538,15 +447,12 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	const RmgrDescData *desc = &RmgrDescTable[XLogRecGetRmid(record)];
 	uint32		rec_len;
 	uint32		fpi_len;
-	RelFileNode rnode;
-	ForkNumber	forknum;
-	BlockNumber blk;
-	int			block_id;
 	uint8		info = XLogRecGetInfo(record);
 	XLogRecPtr	xl_prev = XLogRecGetPrev(record);
 	StringInfoData s;
+	char	delim = {'\n'};
 
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
+	XLogRecGetLen(record, &rec_len, &fpi_len);
 
 	printf("rmgr: %-11s len (rec/tot): %6u/%6u, tx: %10u, lsn: %X/%08X, prev %X/%08X, ",
 		   desc->rm_name,
@@ -564,93 +470,11 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	initStringInfo(&s);
 	desc->rm_desc(&s, record);
 	printf("%s", s.data);
-	pfree(s.data);
-
-	if (!config->bkp_details)
-	{
-		/* print block references (short format) */
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			if (forknum != MAIN_FORKNUM)
-				printf(", blkref #%d: rel %u/%u/%u fork %s blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   forkNames[forknum],
-					   blk);
-			else
-				printf(", blkref #%d: rel %u/%u/%u blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				if (XLogRecBlockImageApply(record, block_id))
-					printf(" FPW");
-				else
-					printf(" FPW for WAL verification");
-			}
-		}
-		putchar('\n');
-	}
-	else
-	{
-		/* print block references (detailed format) */
-		putchar('\n');
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			printf("\tblkref #%d: rel %u/%u/%u fork %s blk %u",
-				   block_id,
-				   rnode.spcNode, rnode.dbNode, rnode.relNode,
-				   forkNames[forknum],
-				   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
 
-				if (BKPIMAGE_COMPRESSED(bimg_info))
-				{
-					const char *method;
-
-					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
-						method = "pglz";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
-						method = "lz4";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
-						method = "zstd";
-					else
-						method = "unknown";
-
-					printf(" (FPW%s); hole: offset: %u, length: %u, "
-						   "compression saved: %u, method: %s",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length,
-						   BLCKSZ -
-						   XLogRecGetBlock(record, block_id)->hole_length -
-						   XLogRecGetBlock(record, block_id)->bimg_len,
-						   method);
-				}
-				else
-				{
-					printf(" (FPW%s); hole: offset: %u, length: %u",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length);
-				}
-			}
-			putchar('\n');
-		}
-	}
+	resetStringInfo(&s);
+	XLogRecGetBlockRefInfo(record, &delim, NULL, config->bkp_details, &s);
+	printf("%s", s.data);
+	pfree(s.data);
 }
 
 /*
@@ -698,7 +522,7 @@ XLogDumpStatsRow(const char *name,
  * Display summary statistics about the records seen so far.
  */
 static void
-XLogDumpDisplayStats(XLogDumpConfig *config, XLogDumpStats *stats)
+XLogDumpDisplayStats(XLogDumpConfig *config, XLogStats *stats)
 {
 	int			ri,
 				rj;
@@ -859,7 +683,7 @@ main(int argc, char **argv)
 	XLogReaderState *xlogreader_state;
 	XLogDumpPrivate private;
 	XLogDumpConfig config;
-	XLogDumpStats stats;
+	XLogStats stats;
 	XLogRecord *record;
 	XLogRecPtr	first_record;
 	char	   *waldir = NULL;
@@ -913,7 +737,7 @@ main(int argc, char **argv)
 
 	memset(&private, 0, sizeof(XLogDumpPrivate));
 	memset(&config, 0, sizeof(XLogDumpConfig));
-	memset(&stats, 0, sizeof(XLogDumpStats));
+	memset(&stats, 0, sizeof(XLogStats));
 
 	private.timeline = 1;
 	private.startptr = InvalidXLogRecPtr;
@@ -1289,7 +1113,7 @@ main(int argc, char **argv)
 		{
 			if (config.stats == true)
 			{
-				XLogDumpCountRecord(&config, &stats, xlogreader_state);
+				XLogRecStoreStats(&stats, xlogreader_state);
 				stats.endptr = xlogreader_state->EndRecPtr;
 			}
 			else
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 0e94833129..d7c35c37c4 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -329,6 +329,11 @@ extern XLogRecPtr RequestXLogSwitch(bool mark_unimportant);
 
 extern void GetOldestRestartPoint(XLogRecPtr *oldrecptr, TimeLineID *oldtli);
 
+extern void XLogRecGetBlockRefInfo(XLogReaderState *record,
+								   char *delimiter, uint32 *fpi_len,
+								   bool detailed_format,
+								   StringInfo blk_ref);
+
 /*
  * Exported for the functions in timeline.c and xlogarchive.c.  Only valid
  * in the startup process.
diff --git a/src/include/access/xlogstats.h b/src/include/access/xlogstats.h
new file mode 100644
index 0000000000..36d833f82b
--- /dev/null
+++ b/src/include/access/xlogstats.h
@@ -0,0 +1,40 @@
+/*-------------------------------------------------------------------------
+ *
+ * xlogstats.h
+ *		Definitions for WAL Statitstics
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		src/include/access/xlogstats.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef XLOGSTATS_H
+#define XLOGSTATS_H
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64	count;
+	uint64	rec_len;
+	uint64	fpi_len;
+} XLogRecStats;
+
+typedef struct XLogStats
+{
+	uint64	count;
+#ifdef FRONTEND
+	XLogRecPtr	startptr;
+	XLogRecPtr	endptr;
+#endif
+	XLogRecStats    rmgr_stats[RM_NEXT_ID];
+	XLogRecStats    record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+} XLogStats;
+
+extern void XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+						  uint32 *fpi_len);
+extern void XLogRecStoreStats(XLogStats *stats, XLogReaderState *record);
+
+#endif							/* XLOGSTATS_H */
-- 
2.25.1

v16-0002-pg_walinspect.patchapplication/octet-stream; name=v16-0002-pg_walinspect.patchDownload
From d0ce90d0d0c73ee7abc1ec017c6772f5e4a8d3e6 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sat, 26 Mar 2022 04:55:12 +0000
Subject: [PATCH v16] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 132 ++++
 contrib/pg_walinspect/pg_walinspect.c        | 654 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  13 +-
 src/backend/access/transam/xlogutils.c       |  33 +
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/access/xlogutils.h               |   4 +
 13 files changed, 872 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 332b486ecc..bbf220407b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -41,6 +41,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..9b192dd2c1
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,132 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_wal_record()
+--
+CREATE FUNCTION pg_get_wal_record(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT record_length int4,
+    OUT record bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text,
+    OUT data_length int4,
+    OUT data bytea
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT resource_manager text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn) TO pg_read_server_files;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..adf7fc5890
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,654 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogstats.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(pg_get_wal_record);
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats_till_end_of_wal);
+
+typedef void (*GetWALDetailsCB) (FunctionCallInfo fcinfo,
+								 XLogRecPtr start_lsn,
+								 XLogRecPtr end_lsn);
+
+static bool IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static Datum GetWALRecordInternal(FunctionCallInfo fcinfo, Datum *values,
+								  bool *nulls, uint32 ncols, bool get_info);
+static void GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							  Datum *values, bool *nulls, uint32 ncols);
+static void GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+							  GetWALDetailsCB wal_details_cb);
+static void GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+							  XLogRecPtr end_lsn);
+static void GetXLogSummaryStats(XLogStats * stats, ReturnSetInfo *rsinfo,
+								Datum *values, bool *nulls, uint32 ncols);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						XLogRecPtr end_lsn);
+
+/*
+ * Check if the given LSN is in future. Also, return the LSN up to which the
+ * server has WAL.
+ */
+static bool
+IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn)
+{
+	/*
+	 * We determine the current LSN of the server similar to how page_read
+	 * callback read_local_xlog_page_no_wait does.
+	 */
+	if (!RecoveryInProgress())
+		*curr_lsn = GetFlushRecPtr(NULL);
+	else
+		*curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(*curr_lsn));
+
+	if (lsn >= *curr_lsn)
+		return true;
+
+	return false;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page_no_wait,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get WAL record info.
+ */
+static void
+GetXLogRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				  Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	bytea	*data;
+	char	*main_data;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	initStringInfo(&rec_desc);
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		appendStringInfo(&rec_desc, "UNKNOWN (%x) ", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+	else
+		appendStringInfo(&rec_desc, "%s ", id);
+
+	desc->rm_desc(&rec_desc, record);
+
+	/* Block references. */
+	initStringInfo(&rec_blk_ref);
+	XLogRecGetBlockRefInfo(record, NULL, &fpi_len, true, &rec_blk_ref);
+
+	main_data_len = XLogRecGetDataLen(record);
+	data = (bytea *) palloc(main_data_len + VARHDRSZ);
+	SET_VARSIZE(data, main_data_len + VARHDRSZ);
+	main_data = VARDATA(data);
+	memcpy(main_data, XLogRecGetData(record), main_data_len);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = PointerGetDatum(data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record data or info.
+ */
+Datum
+GetWALRecordInternal(FunctionCallInfo fcinfo, Datum *values, bool *nulls,
+					 uint32 ncols, bool get_info)
+{
+	XLogRecPtr	lsn;
+	XLogRecPtr	curr_lsn;
+	XLogRecord *record;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	HeapTuple	tuple;
+	Datum	result;
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (IsFutureLSN(lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	record = ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, ncols);
+	MemSet(nulls, 0, ncols);
+
+	if (get_info)
+	{
+		GetXLogRecordInfo(xlogreader, first_record, values, nulls,
+						  ncols);
+	}
+	else
+	{
+		bytea	*rec;
+		uint32	rec_len;
+		char	*rec_data;
+		int	i = 0;
+
+		rec_len = XLogRecGetTotalLen(xlogreader);
+
+		Assert(rec_len > 0);
+
+		rec = (bytea *) palloc(rec_len + VARHDRSZ);
+		SET_VARSIZE(rec, rec_len + VARHDRSZ);
+		rec_data = VARDATA(rec);
+
+		memcpy(rec_data, record, rec_len);
+
+		values[i++] = LSNGetDatum(first_record);
+		values[i++] = LSNGetDatum(xlogreader->EndRecPtr - 1);
+		values[i++] = LSNGetDatum(XLogRecGetPrev(xlogreader));
+		values[i++] = UInt32GetDatum(rec_len);
+		values[i++] = PointerGetDatum(rec);
+
+		Assert(i == ncols);
+	}
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	return result;
+}
+
+/*
+ * Get WAL record.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_COLS 5
+	Datum	result;
+	Datum	values[PG_GET_WAL_RECORD_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_COLS];
+
+	result = GetWALRecordInternal(fcinfo, values, nulls,
+								  PG_GET_WAL_RECORD_COLS,
+								  false);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_COLS
+}
+
+/*
+ * Get WAL record info and data.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	Datum	result;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+
+	result = GetWALRecordInternal(fcinfo, values, nulls,
+								  PG_GET_WAL_RECORD_INFO_COLS,
+								  true);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get WAL details such as record info, stats using the passed in callback.
+ */
+static void
+GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+				  GetWALDetailsCB wal_details_cb)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecPtr	curr_lsn;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* If not till end of wal, end_lsn would have been specified. */
+	if (!till_end_of_wal)
+		end_lsn = PG_GETARG_LSN(1);
+
+	if (IsFutureLSN(start_lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	if (!till_end_of_wal && end_lsn >= curr_lsn)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+	else if (till_end_of_wal)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up to
+		 * "end".
+		 */
+		end_lsn = curr_lsn - 1;
+	}
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	wal_details_cb(fcinfo, start_lsn, end_lsn);
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+				  XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+		{
+			GetXLogRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+							  PG_GET_WAL_RECORDS_INFO_COLS);
+
+			tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+								 values, nulls);
+		}
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get info and data of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double	n_pct,
+			rec_len_pct,
+			fpi_len_pct,
+			tot_len_pct;
+	int	i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogStats *stats, ReturnSetInfo *rsinfo,
+					Datum *values, bool *nulls, uint32 ncols)
+{
+	uint64	total_count = 0;
+	uint64	total_rec_len = 0;
+	uint64	total_fpi_len = 0;
+	uint64	total_len = 0;
+	int	ri;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		count = stats->rmgr_stats[ri].count;
+		rec_len = stats->rmgr_stats[ri].rec_len;
+		fpi_len = stats->rmgr_stats[ri].fpi_len;
+		tot_len = rec_len + fpi_len;
+
+		FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+						 total_rec_len, fpi_len, total_fpi_len, tot_len,
+						 total_len, values, nulls, ncols);
+
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+							 values, nulls);
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+			XLogRecPtr end_lsn)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogStats stats;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_STATS_COLS];
+	bool	nulls[PG_GET_WAL_STATS_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+			XLogRecStoreStats(&stats, xlogreader);
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, rsinfo, values, nulls,
+						PG_GET_WAL_STATS_COLS);
+
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get stats of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWalStats);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get stats of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWalStats);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index e437c42992..585c94c488 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -1320,13 +1320,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1447,6 +1440,12 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index 8c1b8216be..4a4ef8a338 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -85,6 +85,10 @@ typedef struct xl_missing_dir
 
 static HTAB *missing_dir_tab = NULL;
 
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal);
 
 /*
  * Keep track of a directory that wasn't found while replaying database
@@ -1008,6 +1012,31 @@ wal_segment_close(XLogReaderState *state)
 int
 read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 					 int reqLen, XLogRecPtr targetRecPtr, char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, true);
+}
+
+/*
+ * Same as read_local_xlog_page except that it doesn't wait for future WAL
+ * to be available.
+ */
+int
+read_local_xlog_page_no_wait(XLogReaderState *state, XLogRecPtr targetPagePtr,
+							 int reqLen, XLogRecPtr targetRecPtr,
+							 char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, false);
+}
+
+/*
+ * Implementation of read_local_xlog_page and its no wait version.
+ */
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal)
 {
 	XLogRecPtr	read_upto,
 				loc;
@@ -1063,6 +1092,10 @@ read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 			if (loc <= read_upto)
 				break;
 
+			/* If asked, let's not wait for future WAL. */
+			if (!wait_for_wal)
+				break;
+
 			CHECK_FOR_INTERRUPTS();
 			pg_usleep(1000L);
 		}
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index f314d33ebf..cad5b3594a 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -27,6 +27,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 09f6464331..3e644372f9 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index d7c35c37c4..2985c75361 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index f4388cc9be..b4c7d93787 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -340,9 +340,7 @@ extern void XLogReaderSetDecodeBuffer(XLogReaderState *state,
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Return values from XLogPageReadCB. */
 typedef enum XLogPageReadResult
diff --git a/src/include/access/xlogutils.h b/src/include/access/xlogutils.h
index 8d48f003b0..40cdb1b947 100644
--- a/src/include/access/xlogutils.h
+++ b/src/include/access/xlogutils.h
@@ -96,6 +96,10 @@ extern void FreeFakeRelcacheEntry(Relation fakerel);
 extern int	read_local_xlog_page(XLogReaderState *state,
 								 XLogRecPtr targetPagePtr, int reqLen,
 								 XLogRecPtr targetRecPtr, char *cur_page);
+extern int	read_local_xlog_page_no_wait(XLogReaderState *state,
+										 XLogRecPtr targetPagePtr, int reqLen,
+										 XLogRecPtr targetRecPtr,
+										 char *cur_page);
 extern void wal_segment_open(XLogReaderState *state,
 							 XLogSegNo nextSegNo,
 							 TimeLineID *tli_p);
-- 
2.25.1

v16-0003-pg_walinspect-tests.patchapplication/octet-stream; name=v16-0003-pg_walinspect-tests.patchDownload
From 0de6ed668e4c05d534360e0d14b8cd91fd1e52bd Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sat, 26 Mar 2022 04:56:17 +0000
Subject: [PATCH v16] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 216 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 146 ++++++++++++
 2 files changed, 362 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..7445c7dfde
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,216 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+-- Functions accessible by users with role pg_read_server_files
+GRANT pg_read_server_files TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..3a97ef7153
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,146 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- no
+
+-- Functions accessible by users with role pg_read_server_files
+
+GRANT pg_read_server_files TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats_till_end_of_wal(pg_lsn) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v16-0004-pg_walinspect-docs.patchapplication/octet-stream; name=v16-0004-pg_walinspect-docs.patchDownload
From 7e99df3630bffe9dc9c4164f2416a13d759dc4ac Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Sat, 26 Mar 2022 04:57:04 +0000
Subject: [PATCH v16] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 234 +++++++++++++++++++++++++++++++++
 3 files changed, 236 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index 1e42ce1a7f..4e7b87a42f 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -131,6 +131,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index fd853af01f..34c19c80f1 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -147,6 +147,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..8ab13c38c4
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,234 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_read_server_files</literal> role. Access may be granted by
+  superusers to others using <command>GRANT</command>.
+ </para>
+    
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record(in_lsn pg_lsn,
+                        start_lsn OUT pg_lsn,
+                        end_lsn OUT pg_lsn,
+                        record_length OUT int4,
+                        record OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets raw WAL record data of a given LSN. This function emits an error if
+      a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text,
+                             data_length OUT int4,
+                             data OUT bytea)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of a given LSN. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>in_lsn</replaceable> is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text,
+                              data_length OUT int4,
+                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_length, fpi_length, block_ref, data_length from pg_get_wal_records_info('0/1401790', '0/1401D88');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager | record_length | fpi_length |                     block_ref                      | data_length 
+-----------+-----------+-----------+-----+------------------+---------------+------------+----------------------------------------------------+-------------
+ 0/14018B0 | 0/14018EF | 0/1401788 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/14018F0 | 0/1401A17 | 0/14018B0 | 673 | Heap             |           290 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401A18 | 0/1401A57 | 0/14018F0 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401A58 | 0/1401B47 | 0/1401A18 | 673 | Heap             |           234 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401B48 | 0/1401B87 | 0/1401A58 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401B88 | 0/1401C67 | 0/1401B48 | 673 | Heap             |           222 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+ 0/1401C68 | 0/1401CA7 | 0/1401B88 | 673 | Btree            |            64 |          0 |         blkref #0: rel 1663/1/2696 fork main blk 1 |           2
+ 0/1401CA8 | 0/1401D87 | 0/1401C68 | 673 | Heap             |           217 |          0 |         blkref #0: rel 1663/1/2619 fork main blk 1 |           3
+(8 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info_till_end_of_wal(start_lsn pg_lsn,
+                                              start_lsn OUT pg_lsn,
+                                              end_lsn OUT pg_lsn,
+                                              prev_lsn OUT pg_lsn,
+                                              xid OUT xid,
+                                              resource_manager OUT text,
+                                              record_length OUT int4,
+                                              fpi_length OUT int4,
+                                              description OUT text,
+                                              block_ref OUT text,
+                                              data_length OUT int4,
+                                              data OUT bytea)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_records_info()</function>
+      except that it gets information of all the valid WAL records from 
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       resource_manager OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4,
+                       combined_size OUT int8,
+                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/13E80C0', '0/1600000') where count > 0;
+ resource_manager | count | count_percentage | record_size | record_size_percentage | fpi_size | fpi_size_percentage | combined_size | combined_size_percentage 
+------------------+-------+------------------+-------------+------------------------+----------+---------------------+---------------+--------------------------
+ XLOG             |     6 |      0.032156065 |         600 |          1.3965058e-05 |        0 |                   0 |           600 |            1.3963012e-05
+ Transaction      |    61 |       0.32691997 |       16399 |          0.00038168833 |        0 |                   0 |         16399 |             0.0003816324
+ Storage          |     1 |      0.005359344 |          42 |           9.775541e-07 |        0 |                   0 |            42 |             9.774109e-07
+ Database         |     2 |      0.010718688 |          84 |          1.9551082e-06 |        0 |                   0 |            84 |            1.9548218e-06
+ Standby          |   147 |       0.78782356 |       12726 |           0.0002961989 |        0 |                   0 |         12726 |             0.0002961555
+ Heap2            |   804 |        4.3089128 |      190283 |           0.0044288556 |   374096 |            59.41106 |        564379 |              0.013134051
+ Heap             | 16790 |         89.98338 |     1191835 |            0.027740076 |   157528 |            25.01739 |       1349363 |              0.031401955
+ Btree            |   848 |        4.5447235 |  4295025395 |               99.96713 |    98050 |           15.571549 |    4295123445 |                 99.95477
+(8 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats_till_end_of_wal(start_lsn pg_lsn,
+                                       resource_manager OUT text,
+                                       count OUT int8,
+                                       count_percentage OUT float4,
+                                       record_length OUT int8,
+                                       record_length_percentage OUT float4,
+                                       fpi_length OUT int8,
+                                       fpi_length_percentage OUT float4,
+                                       combined_size OUT int8,
+                                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_stats()</function> except
+      that it gets stats of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#82Jeff Davis
pgsql@j-davis.com
In reply to: Bharath Rupireddy (#81)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Sat, 2022-03-26 at 10:31 +0530, Bharath Rupireddy wrote:

Attaching v16 patch-set, only change in v16-0002-pg_walinspect.patch,
others remain the same.

I looked more closely at this patch.

* It seems that pg_get_wal_record() is not returning the correct raw
data for the record. I tested with pg_logical_emit_message, and the
message isn't there. pg_get_wal_record_info() uses XLogRecordGetData(),
which seems closer to what I expect.

* I'm a little unclear on the purpose of pg_get_wal_record(). What does
it offer that the other functions don't?

* I don't think we need the stats at all. We can run GROUP BY queries
on the results of pg_get_wal_records_info().

* Include the xlinfo portion of the wal record in addition to the rmgr,
like pg_waldump --stats=record shows. That way we can GROUP BY that as
well.

* I don't think we need the changes to xlogutils.c. You calculate the
end pointer based on the flush pointer, anyway, so we should never need
to wait (and when I take it out, the tests still pass).

I think we can radically simplify it without losing functionality,
unless I'm missing something.

1. Eliminate pg_get_wal_record(),
pg_get_wal_records_info_till_end_of_wal(), pg_get_wal_stats(),
pg_get_wal_stats_till_end_of_wal().

2. Rename pg_get_wal_record_info -> pg_get_wal_record

3. Rename pg_get_wal_records_info -> pg_get_wal_records

4. For pg_get_wal_records, if end_lsn is NULL, read until the end of
WAL.

5. For pg_get_wal_record and pg_get_wal_records, also return the xlinfo
using rm_identify() if available.

6. Remove changes to xlogutils.

7. Remove the refactor to pull the stats out to a separate file,
because stats aren't needed.

8. With only two functions in the API, it may even make sense to just
make it a part of postgres rather than a separate module.

However, I'm a little behind on this discussion thread, so perhaps I'm
missing some important context. I'll try to catch up soon.

Regards,
Jeff Davis

#83Andres Freund
andres@anarazel.de
In reply to: Jeff Davis (#82)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

Hi,

On 2022-04-01 16:35:38 -0700, Jeff Davis wrote:

* I don't think we need the stats at all. We can run GROUP BY queries
on the results of pg_get_wal_records_info().

It's well over an order of magnitude slower. And I don't see how that can be
avoided. That makes it practically useless.

See numbers at the bottom of
/messages/by-id/CALj2ACUvU2fGLw7keEpxZhGAoMQ9vrCPX-13hexQPoR+QRbuOw@mail.gmail.com

Greetings,

Andres Freund

#84Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Jeff Davis (#82)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Sat, Apr 2, 2022 at 5:05 AM Jeff Davis <pgsql@j-davis.com> wrote:

On Sat, 2022-03-26 at 10:31 +0530, Bharath Rupireddy wrote:

Attaching v16 patch-set, only change in v16-0002-pg_walinspect.patch,
others remain the same.

I looked more closely at this patch.

Thanks Jeff for reviewing this.

* It seems that pg_get_wal_record() is not returning the correct raw
data for the record. I tested with pg_logical_emit_message, and the
message isn't there. pg_get_wal_record_info() uses XLogRecordGetData(),
which seems closer to what I expect.

* I'm a little unclear on the purpose of pg_get_wal_record(). What does
it offer that the other functions don't?

My intention is to return the overall undecoded WAL record [5]/* * The overall layout of an XLOG record is: * Fixed-size header (XLogRecord struct) * XLogRecordBlockHeader struct * XLogRecordBlockHeader struct * ... * XLogRecordDataHeader[Short|Long] struct * block data * block data * ... * main data i.e.
the data starting from XLogReadRecord's output [6]XLogRecord * XLogReadRecord(XLogReaderState *state, char **errormsg) { decoded = XLogNextRecord(state, errormsg); if (decoded) { /* * This function returns a pointer to the record's header, not the * actual decoded record. The caller will access the decoded record * through the XLogRecGetXXX() macros, which reach the decoded * recorded as xlogreader->record. */ Assert(state->record == decoded); return &decoded->header; } till length
XLogRecGetTotalLen(xlogreader);. Please see [7]/messages/by-id/20220322180006.hgbsoldgbljyrcm7@alap3.anarazel.de, where Andres agreed
to have this function, I also mentioned a possible use-case there.

pg_get_wal_record_info returns the main data of the WAL record
(xl_heap_delete, xl_heap_insert, xl_heap_multi_insert, xl_heap_update
and so on).

* I don't think we need the stats at all. We can run GROUP BY queries
on the results of pg_get_wal_records_info().

As identified in [1]/messages/by-id/CALj2ACUvU2fGLw7keEpxZhGAoMQ9vrCPX-13hexQPoR+QRbuOw@mail.gmail.com, SQL-version of stats function is way slower in
normal cases, hence it was agreed (by Andres, Kyotaro-san and myself)
to have a C-function for stats.

* Include the xlinfo portion of the wal record in addition to the rmgr,
like pg_waldump --stats=record shows. That way we can GROUP BY that as
well.

5. For pg_get_wal_record and pg_get_wal_records, also return the xlinfo
using rm_identify() if available.

Yes, that's already part of the description column (much like
pg_waldump does) and the users can still do it with GROUP BY and
HAVING clauses, see [4]postgres=# select count(resource_manager), description, from pg_get_wal_records_info('0/14E0568', '0/14F2568') group by description having description like '%INSERT_LEAF%'; count | description -------+--------------------- 7 | INSERT_LEAF off 108 1 | INSERT_LEAF off 111 1 | INSERT_LEAF off 135 1 | INSERT_LEAF off 142 3 | INSERT_LEAF off 143 1 | INSERT_LEAF off 144 1 | INSERT_LEAF off 145 1 | INSERT_LEAF off 146 1 | INSERT_LEAF off 274 1 | INSERT_LEAF off 405 (10 rows).

* I don't think we need the changes to xlogutils.c. You calculate the
end pointer based on the flush pointer, anyway, so we should never need
to wait (and when I take it out, the tests still pass).

6. Remove changes to xlogutils.

As mentioned in [1]/messages/by-id/CALj2ACUvU2fGLw7keEpxZhGAoMQ9vrCPX-13hexQPoR+QRbuOw@mail.gmail.com, read_local_xlog_page_no_wait required because the
functions can still wait in read_local_xlog_page for WAL while finding
the first valid record after the given input LSN (the use case is
simple - just provide input LSN closer to server's current flush LSN,
may be off by 3 or 4 bytes).

Also, I tried to keep the changes minimal with the
read_local_xlog_page_guts static function. IMO, that shouldn't be a
problem.

I think we can radically simplify it without losing functionality,
unless I'm missing something.

1. Eliminate pg_get_wal_record(),
pg_get_wal_records_info_till_end_of_wal(), pg_get_wal_stats(),
pg_get_wal_stats_till_end_of_wal().

4. For pg_get_wal_records, if end_lsn is NULL, read until the end of
WAL.

It's pretty much clear to the users with till_end_of_wal functions
instead of cooking many things into the same functions with default
values for input LSNs as NULL which also requires the functions to be
"CALLED ON NULL INPUT" types which isn't good. This was also suggested
by Andres, see [2]/messages/by-id/20220322180006.hgbsoldgbljyrcm7@alap3.anarazel.de, and I agree with it.

2. Rename pg_get_wal_record_info -> pg_get_wal_record

3. Rename pg_get_wal_records_info -> pg_get_wal_records

As these functions aren't returning the WAL record data, but info
about it (decoded data), I would like to retain the function names
as-is.

8. With only two functions in the API, it may even make sense to just
make it a part of postgres rather than a separate module.

As said above, I would like to have till_end_of_wal versions. Firstly,
pg_walinspect functions may not be needed by everyone, the extension
provides a way for the users to install if required. Also, many
hackers have suggested new functions [3]There are many functions we can add to pg_walinspect - functions with wait mode for future WAL, WAL parsing, function to return all the WAL record info/stats given a WAL file name, functions to return WAL info/stats from historic timelines as well, function to see if the given WAL file is valid and so on., but, right now the idea is
to get pg_walinspect onboard with simple-yet-useful functions and then
think of extending it with new functions later.

[1]: /messages/by-id/CALj2ACUvU2fGLw7keEpxZhGAoMQ9vrCPX-13hexQPoR+QRbuOw@mail.gmail.com
[2]: /messages/by-id/20220322180006.hgbsoldgbljyrcm7@alap3.anarazel.de
[3]: There are many functions we can add to pg_walinspect - functions with wait mode for future WAL, WAL parsing, function to return all the WAL record info/stats given a WAL file name, functions to return WAL info/stats from historic timelines as well, function to see if the given WAL file is valid and so on.
with wait mode for future WAL, WAL parsing, function to return all the
WAL record info/stats given a WAL file name, functions to return WAL
info/stats from historic timelines as well, function to see if the
given WAL file is valid and so on.
[4]: postgres=# select count(resource_manager), description, from pg_get_wal_records_info('0/14E0568', '0/14F2568') group by description having description like '%INSERT_LEAF%'; count | description -------+--------------------- 7 | INSERT_LEAF off 108 1 | INSERT_LEAF off 111 1 | INSERT_LEAF off 135 1 | INSERT_LEAF off 142 3 | INSERT_LEAF off 143 1 | INSERT_LEAF off 144 1 | INSERT_LEAF off 145 1 | INSERT_LEAF off 146 1 | INSERT_LEAF off 274 1 | INSERT_LEAF off 405 (10 rows)
postgres=# select count(resource_manager), description, from
pg_get_wal_records_info('0/14E0568', '0/14F2568') group by description
having description like '%INSERT_LEAF%';
count | description
-------+---------------------
7 | INSERT_LEAF off 108
1 | INSERT_LEAF off 111
1 | INSERT_LEAF off 135
1 | INSERT_LEAF off 142
3 | INSERT_LEAF off 143
1 | INSERT_LEAF off 144
1 | INSERT_LEAF off 145
1 | INSERT_LEAF off 146
1 | INSERT_LEAF off 274
1 | INSERT_LEAF off 405
(10 rows)

[5]: /* * The overall layout of an XLOG record is: * Fixed-size header (XLogRecord struct) * XLogRecordBlockHeader struct * XLogRecordBlockHeader struct * ... * XLogRecordDataHeader[Short|Long] struct * block data * block data * ... * main data
/*
* The overall layout of an XLOG record is:
* Fixed-size header (XLogRecord struct)
* XLogRecordBlockHeader struct
* XLogRecordBlockHeader struct
* ...
* XLogRecordDataHeader[Short|Long] struct
* block data
* block data
* ...
* main data

[6]: XLogRecord * XLogReadRecord(XLogReaderState *state, char **errormsg) { decoded = XLogNextRecord(state, errormsg); if (decoded) { /* * This function returns a pointer to the record's header, not the * actual decoded record. The caller will access the decoded record * through the XLogRecGetXXX() macros, which reach the decoded * recorded as xlogreader->record. */ Assert(state->record == decoded); return &decoded->header; }
XLogRecord *
XLogReadRecord(XLogReaderState *state, char **errormsg)
{
decoded = XLogNextRecord(state, errormsg);
if (decoded)
{
/*
* This function returns a pointer to the record's header, not the
* actual decoded record. The caller will access the decoded record
* through the XLogRecGetXXX() macros, which reach the decoded
* recorded as xlogreader->record.
*/
Assert(state->record == decoded);
return &decoded->header;
}

[7]: /messages/by-id/20220322180006.hgbsoldgbljyrcm7@alap3.anarazel.de

Regards,
Bharath Rupireddy.

#85Jeff Davis
pgsql@j-davis.com
In reply to: Bharath Rupireddy (#84)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Mon, 2022-04-04 at 09:15 +0530, Bharath Rupireddy wrote:

My intention is to return the overall undecoded WAL record [5] i.e.
the data starting from XLogReadRecord's output [6] till length
XLogRecGetTotalLen(xlogreader);. Please see [7], where Andres agreed
to have this function, I also mentioned a possible use-case there.

The current patch does not actually do this: it's returning a pointer
into the DecodedXLogRecord struct, which doesn't have the raw bytes of
the WAL record.

To return the raw bytes of the record is not entirely trivial: it seems
we have to look in the decoded record and either find a pointer into
readBuf, or readRecordBuf, depending on whether the record crosses a
boundary or not. If we find a good way to do this I'm fine keeping the
function, but if not, we can leave it for v16.

pg_get_wal_record_info returns the main data of the WAL record
(xl_heap_delete, xl_heap_insert, xl_heap_multi_insert, xl_heap_update
and so on).

We also discussed just removing the main data from the output here.
It's not terribly useful, and could be arbitrarily large. Similar to
how we leave out the backup block data and images.

As identified in [1], SQL-version of stats function is way slower in
normal cases, hence it was agreed (by Andres, Kyotaro-san and myself)
to have a C-function for stats.a pointer into

Now I agree. We should also have an equivalent of "pg_waldump --
stats=record" though, too.

Yes, that's already part of the description column (much like
pg_waldump does) and the users can still do it with GROUP BY and
HAVING clauses, see [4].

I still think an extra column for the results of rm_identify() would
make sense. Not critical, but seems useful.

As mentioned in [1], read_local_xlog_page_no_wait required because
the
functions can still wait in read_local_xlog_page for WAL while
finding
the first valid record after the given input LSN (the use case is
simple - just provide input LSN closer to server's current flush LSN,
may be off by 3 or 4 bytes).

Did you look into using XLogReadAhead() rather than XLogReadRecord()?

It's pretty much clear to the users with till_end_of_wal functions
instead of cooking many things into the same functions with default
values for input LSNs as NULL which also requires the functions to be
"CALLED ON NULL INPUT" types which isn't good. This was also
suggested
by Andres, see [2], and I agree with it.

OK, it's a matter of taste I suppose. I don't have a strong opinion.

2. Rename pg_get_wal_record_info -> pg_get_wal_record

3. Rename pg_get_wal_records_info -> pg_get_wal_records

As these functions aren't returning the WAL record data, but info
about it (decoded data), I would like to retain the function names
as-is.

The name pg_get_wal_records_info bothers me slightly, but I don't have
a better suggestion.

One other thought: functions like pg_logical_emit_message() return an
LSN, but if you feed that into pg_walinspect you get the *next* record.
That makes sense because pg_logical_emit_message() returns the result
of XLogInsertRecord(), which is the end of the last inserted record.
But it can be slightly annoying/confusing. I don't have any particular
suggestion, but maybe it's worth a mention in the docs or something?

Regards,
Jeff Davis

#86Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Jeff Davis (#85)
4 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Apr 6, 2022 at 10:32 AM Jeff Davis <pgsql@j-davis.com> wrote:

On Mon, 2022-04-04 at 09:15 +0530, Bharath Rupireddy wrote:

My intention is to return the overall undecoded WAL record [5] i.e.
the data starting from XLogReadRecord's output [6] till length
XLogRecGetTotalLen(xlogreader);. Please see [7], where Andres agreed
to have this function, I also mentioned a possible use-case there.

The current patch does not actually do this: it's returning a pointer
into the DecodedXLogRecord struct, which doesn't have the raw bytes of
the WAL record.

To return the raw bytes of the record is not entirely trivial: it seems
we have to look in the decoded record and either find a pointer into
readBuf, or readRecordBuf, depending on whether the record crosses a
boundary or not. If we find a good way to do this I'm fine keeping the
function, but if not, we can leave it for v16.

With no immediate use of raw WAL data without a WAL record parsing
function, I'm dropping that function for now.

pg_get_wal_record_info returns the main data of the WAL record
(xl_heap_delete, xl_heap_insert, xl_heap_multi_insert, xl_heap_update
and so on).

We also discussed just removing the main data from the output here.
It's not terribly useful, and could be arbitrarily large. Similar to
how we leave out the backup block data and images.

Done.

As identified in [1], SQL-version of stats function is way slower in
normal cases, hence it was agreed (by Andres, Kyotaro-san and myself)
to have a C-function for stats.a pointer into

Now I agree. We should also have an equivalent of "pg_waldump --
stats=record" though, too.

Added a parameter per_record (with default being false, emitting
per-rmgr stats) to pg_get_wal_stats and
pg_get_wal_stats_till_end_of_wal, when set returns per-record stats,
much like pg_waldump.

Yes, that's already part of the description column (much like
pg_waldump does) and the users can still do it with GROUP BY and
HAVING clauses, see [4].

I still think an extra column for the results of rm_identify() would
make sense. Not critical, but seems useful.

Added rm_identify as record_type column in pg_get_wal_record_info,
pg_get_wal_records_info, pg_get_wal_record_info_till_end_of_wal.
Removed the rm_identify from the description column as it's
unnecessary now here.

As mentioned in [1], read_local_xlog_page_no_wait required because
the
functions can still wait in read_local_xlog_page for WAL while
finding
the first valid record after the given input LSN (the use case is
simple - just provide input LSN closer to server's current flush LSN,
may be off by 3 or 4 bytes).

Did you look into using XLogReadAhead() rather than XLogReadRecord()?

I don't think XLogReadAhead will help either, as it calls page_read
callback, XLogReadAhead->XLogDecodeNextRecord->ReadPageInternal->page_read->read_local_xlog_page
(which again waits for future WAL).

Per our internal discussion, I'm keeping the
read_local_xlog_page_no_wait as it offers a better solution without
much code duplication.

The name pg_get_wal_records_info bothers me slightly, but I don't have
a better suggestion.

IMO, pg_get_wal_records_info looks okay, hence didn't change it.

One other thought: functions like pg_logical_emit_message() return an
LSN, but if you feed that into pg_walinspect you get the *next* record.
That makes sense because pg_logical_emit_message() returns the result
of XLogInsertRecord(), which is the end of the last inserted record.
But it can be slightly annoying/confusing. I don't have any particular
suggestion, but maybe it's worth a mention in the docs or something?

Yes, all the pg_walinspect functions would find the next valid WAL
record to the input/start LSN and start returning the details from
then.

IMO, the descriptions of these functions have already specified it:

pg_get_wal_record_info
Gets WAL record information of a given LSN. If the given LSN isn't
containing a valid WAL record, it gives the information of the next
available valid WAL record. This function emits an error if a future (the

all other functions say this:
Gets information/statistics of all the valid WAL records between/from

Attaching v17 patch-set with the above review comments addressed.
Please have a look at it.

Regards,
Bharath Rupireddy.

Attachments:

v17-0001-Refactor-pg_waldump-code.patchapplication/octet-stream; name=v17-0001-Refactor-pg_waldump-code.patchDownload
From bdb838c0538b30b798e58424fb06498a9acb008d Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 6 Apr 2022 02:56:20 +0000
Subject: [PATCH v17] Refactor pg_waldump code

This patch puts some generic chunks of pg_waldump's code into
separate reusable functions in xlogdesc.c and xlogstats.c, a new
file along xlogstats.h introduced for placing WAL stats and
structures.

This way, other modules can reuse these common functions.
---
 src/backend/access/rmgrdesc/xlogdesc.c | 125 ++++++++++++++++
 src/backend/access/transam/Makefile    |   1 +
 src/backend/access/transam/xlogstats.c |  93 ++++++++++++
 src/bin/pg_waldump/.gitignore          |   1 +
 src/bin/pg_waldump/Makefile            |   8 +-
 src/bin/pg_waldump/pg_waldump.c        | 198 ++-----------------------
 src/include/access/xlog_internal.h     |   5 +
 src/include/access/xlogstats.h         |  40 +++++
 8 files changed, 282 insertions(+), 189 deletions(-)
 create mode 100644 src/backend/access/transam/xlogstats.c
 create mode 100644 src/include/access/xlogstats.h

diff --git a/src/backend/access/rmgrdesc/xlogdesc.c b/src/backend/access/rmgrdesc/xlogdesc.c
index e7452af679..429e5dcd5b 100644
--- a/src/backend/access/rmgrdesc/xlogdesc.c
+++ b/src/backend/access/rmgrdesc/xlogdesc.c
@@ -200,3 +200,128 @@ xlog_identify(uint8 info)
 
 	return id;
 }
+
+/*
+ * Returns a string giving information about all the blocks in an
+ * XLogRecord.
+ */
+void
+XLogRecGetBlockRefInfo(XLogReaderState *record, char *delimiter,
+					   uint32 *fpi_len, bool detailed_format,
+					   StringInfo buf)
+{
+	int	block_id;
+
+	Assert(record != NULL);
+
+	if (detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		RelFileNode rnode = {InvalidOid, InvalidOid, InvalidOid};
+		ForkNumber	forknum = InvalidForkNumber;
+		BlockNumber blk = InvalidBlockNumber;
+
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (detailed_format)
+		{
+			/* Get block references in detailed format. */
+
+			appendStringInfo(buf,
+							 "\tblkref #%d: rel %u/%u/%u fork %s blk %u",
+							 block_id,
+							 rnode.spcNode, rnode.dbNode, rnode.relNode,
+							 forkNames[forknum],
+							 blk);
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
+
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (BKPIMAGE_COMPRESSED(bimg_info))
+				{
+					const char *method;
+
+					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+						method = "pglz";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+						method = "lz4";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
+						method = "zstd";
+					else
+						method = "unknown";
+
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u, "
+									 "compression saved: %u, method: %s",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length,
+									 BLCKSZ -
+									 XLogRecGetBlock(record, block_id)->hole_length -
+									 XLogRecGetBlock(record, block_id)->bimg_len,
+									 method);
+				}
+				else
+				{
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length);
+				}
+			}
+		}
+		else
+		{
+			/* Get block references in short format. */
+
+			if (forknum != MAIN_FORKNUM)
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u fork %s blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 forkNames[forknum],
+								 blk);
+			}
+			else
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 blk);
+			}
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (XLogRecBlockImageApply(record, block_id))
+					appendStringInfo(buf, " FPW");
+				else
+					appendStringInfo(buf, " FPW for WAL verification");
+			}
+		}
+
+		if (detailed_format && delimiter != NULL)
+			appendStringInfoChar(buf, '\n');
+	}
+
+	if (!detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+}
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 79314c69ab..071f3dbe0f 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -33,6 +33,7 @@ OBJS = \
 	xloginsert.o \
 	xlogreader.o \
 	xlogrecovery.o \
+	xlogstats.o \
 	xlogutils.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/transam/xlogstats.c b/src/backend/access/transam/xlogstats.c
new file mode 100644
index 0000000000..aff3069ecb
--- /dev/null
+++ b/src/backend/access/transam/xlogstats.c
@@ -0,0 +1,93 @@
+/*-------------------------------------------------------------------------
+ *
+ * xlogstats.c
+ *		Functions for WAL Statitstics
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		src/backend/access/transam/xlogstats.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlogreader.h"
+#include "access/xlogstats.h"
+
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+void
+XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+			  uint32 *fpi_len)
+{
+	int	block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+void
+XLogRecStoreStats(XLogStats *stats, XLogReaderState *record)
+{
+	RmgrId	rmid;
+	uint8	recid;
+	uint32	rec_len;
+	uint32	fpi_len;
+
+	Assert(stats != NULL && record != NULL);
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	XLogRecGetLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics */
+
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
diff --git a/src/bin/pg_waldump/.gitignore b/src/bin/pg_waldump/.gitignore
index 3be00a8b61..dabb6e34b6 100644
--- a/src/bin/pg_waldump/.gitignore
+++ b/src/bin/pg_waldump/.gitignore
@@ -23,6 +23,7 @@
 /xactdesc.c
 /xlogdesc.c
 /xlogreader.c
+/xlogstat.c
 
 # Generated by test suite
 /tmp_check/
diff --git a/src/bin/pg_waldump/Makefile b/src/bin/pg_waldump/Makefile
index 9f333d0c8a..d6459e17c7 100644
--- a/src/bin/pg_waldump/Makefile
+++ b/src/bin/pg_waldump/Makefile
@@ -13,7 +13,8 @@ OBJS = \
 	compat.o \
 	pg_waldump.o \
 	rmgrdesc.o \
-	xlogreader.o
+	xlogreader.o \
+	xlogstats.o
 
 override CPPFLAGS := -DFRONTEND $(CPPFLAGS)
 
@@ -29,6 +30,9 @@ pg_waldump: $(OBJS) | submake-libpgport
 xlogreader.c: % : $(top_srcdir)/src/backend/access/transam/%
 	rm -f $@ && $(LN_S) $< .
 
+xlogstats.c: % : $(top_srcdir)/src/backend/access/transam/%
+	rm -f $@ && $(LN_S) $< .
+
 $(RMGRDESCSOURCES): % : $(top_srcdir)/src/backend/access/rmgrdesc/%
 	rm -f $@ && $(LN_S) $< .
 
@@ -42,7 +46,7 @@ uninstall:
 	rm -f '$(DESTDIR)$(bindir)/pg_waldump$(X)'
 
 clean distclean maintainer-clean:
-	rm -f pg_waldump$(X) $(OBJS) $(RMGRDESCSOURCES) xlogreader.c
+	rm -f pg_waldump$(X) $(OBJS) $(RMGRDESCSOURCES) xlogreader.c xlogstats.c
 	rm -rf tmp_check
 
 check:
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 4cb40d068a..61f06e828f 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -21,6 +21,7 @@
 #include "access/xlog_internal.h"
 #include "access/xlogreader.h"
 #include "access/xlogrecord.h"
+#include "access/xlogstats.h"
 #include "common/fe_memutils.h"
 #include "common/logging.h"
 #include "getopt_long.h"
@@ -66,23 +67,6 @@ typedef struct XLogDumpConfig
 	bool		filter_by_fpw;
 } XLogDumpConfig;
 
-typedef struct Stats
-{
-	uint64		count;
-	uint64		rec_len;
-	uint64		fpi_len;
-} Stats;
-
-#define MAX_XLINFO_TYPES 16
-
-typedef struct XLogDumpStats
-{
-	uint64		count;
-	XLogRecPtr	startptr;
-	XLogRecPtr	endptr;
-	Stats		rmgr_stats[RM_NEXT_ID];
-	Stats		record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
-} XLogDumpStats;
 
 #define fatal_error(...) do { pg_log_fatal(__VA_ARGS__); exit(EXIT_FAILURE); } while(0)
 
@@ -453,81 +437,6 @@ XLogRecordHasFPW(XLogReaderState *record)
 	return false;
 }
 
-/*
- * Calculate the size of a record, split into !FPI and FPI parts.
- */
-static void
-XLogDumpRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
-{
-	int			block_id;
-
-	/*
-	 * Calculate the amount of FPI data in the record.
-	 *
-	 * XXX: We peek into xlogreader's private decoded backup blocks for the
-	 * bimg_len indicating the length of FPI data.
-	 */
-	*fpi_len = 0;
-	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-	{
-		if (XLogRecHasBlockImage(record, block_id))
-			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
-	}
-
-	/*
-	 * Calculate the length of the record as the total length - the length of
-	 * all the block images.
-	 */
-	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
-}
-
-/*
- * Store per-rmgr and per-record statistics for a given record.
- */
-static void
-XLogDumpCountRecord(XLogDumpConfig *config, XLogDumpStats *stats,
-					XLogReaderState *record)
-{
-	RmgrId		rmid;
-	uint8		recid;
-	uint32		rec_len;
-	uint32		fpi_len;
-
-	stats->count++;
-
-	rmid = XLogRecGetRmid(record);
-
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
-
-	/* Update per-rmgr statistics */
-
-	stats->rmgr_stats[rmid].count++;
-	stats->rmgr_stats[rmid].rec_len += rec_len;
-	stats->rmgr_stats[rmid].fpi_len += fpi_len;
-
-	/*
-	 * Update per-record statistics, where the record is identified by a
-	 * combination of the RmgrId and the four bits of the xl_info field that
-	 * are the rmgr's domain (resulting in sixteen possible entries per
-	 * RmgrId).
-	 */
-
-	recid = XLogRecGetInfo(record) >> 4;
-
-	/*
-	 * XACT records need to be handled differently. Those records use the
-	 * first bit of those four bits for an optional flag variable and the
-	 * following three bits for the opcode. We filter opcode out of xl_info
-	 * and use it as the identifier of the record.
-	 */
-	if (rmid == RM_XACT_ID)
-		recid &= 0x07;
-
-	stats->record_stats[rmid][recid].count++;
-	stats->record_stats[rmid][recid].rec_len += rec_len;
-	stats->record_stats[rmid][recid].fpi_len += fpi_len;
-}
-
 /*
  * Print a record to stdout
  */
@@ -538,15 +447,12 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	const RmgrDescData *desc = &RmgrDescTable[XLogRecGetRmid(record)];
 	uint32		rec_len;
 	uint32		fpi_len;
-	RelFileNode rnode;
-	ForkNumber	forknum;
-	BlockNumber blk;
-	int			block_id;
 	uint8		info = XLogRecGetInfo(record);
 	XLogRecPtr	xl_prev = XLogRecGetPrev(record);
 	StringInfoData s;
+	char	delim = {'\n'};
 
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
+	XLogRecGetLen(record, &rec_len, &fpi_len);
 
 	printf("rmgr: %-11s len (rec/tot): %6u/%6u, tx: %10u, lsn: %X/%08X, prev %X/%08X, ",
 		   desc->rm_name,
@@ -564,93 +470,11 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	initStringInfo(&s);
 	desc->rm_desc(&s, record);
 	printf("%s", s.data);
-	pfree(s.data);
-
-	if (!config->bkp_details)
-	{
-		/* print block references (short format) */
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			if (forknum != MAIN_FORKNUM)
-				printf(", blkref #%d: rel %u/%u/%u fork %s blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   forkNames[forknum],
-					   blk);
-			else
-				printf(", blkref #%d: rel %u/%u/%u blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				if (XLogRecBlockImageApply(record, block_id))
-					printf(" FPW");
-				else
-					printf(" FPW for WAL verification");
-			}
-		}
-		putchar('\n');
-	}
-	else
-	{
-		/* print block references (detailed format) */
-		putchar('\n');
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			printf("\tblkref #%d: rel %u/%u/%u fork %s blk %u",
-				   block_id,
-				   rnode.spcNode, rnode.dbNode, rnode.relNode,
-				   forkNames[forknum],
-				   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
 
-				if (BKPIMAGE_COMPRESSED(bimg_info))
-				{
-					const char *method;
-
-					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
-						method = "pglz";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
-						method = "lz4";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
-						method = "zstd";
-					else
-						method = "unknown";
-
-					printf(" (FPW%s); hole: offset: %u, length: %u, "
-						   "compression saved: %u, method: %s",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length,
-						   BLCKSZ -
-						   XLogRecGetBlock(record, block_id)->hole_length -
-						   XLogRecGetBlock(record, block_id)->bimg_len,
-						   method);
-				}
-				else
-				{
-					printf(" (FPW%s); hole: offset: %u, length: %u",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length);
-				}
-			}
-			putchar('\n');
-		}
-	}
+	resetStringInfo(&s);
+	XLogRecGetBlockRefInfo(record, &delim, NULL, config->bkp_details, &s);
+	printf("%s", s.data);
+	pfree(s.data);
 }
 
 /*
@@ -698,7 +522,7 @@ XLogDumpStatsRow(const char *name,
  * Display summary statistics about the records seen so far.
  */
 static void
-XLogDumpDisplayStats(XLogDumpConfig *config, XLogDumpStats *stats)
+XLogDumpDisplayStats(XLogDumpConfig *config, XLogStats *stats)
 {
 	int			ri,
 				rj;
@@ -859,7 +683,7 @@ main(int argc, char **argv)
 	XLogReaderState *xlogreader_state;
 	XLogDumpPrivate private;
 	XLogDumpConfig config;
-	XLogDumpStats stats;
+	XLogStats stats;
 	XLogRecord *record;
 	XLogRecPtr	first_record;
 	char	   *waldir = NULL;
@@ -913,7 +737,7 @@ main(int argc, char **argv)
 
 	memset(&private, 0, sizeof(XLogDumpPrivate));
 	memset(&config, 0, sizeof(XLogDumpConfig));
-	memset(&stats, 0, sizeof(XLogDumpStats));
+	memset(&stats, 0, sizeof(XLogStats));
 
 	private.timeline = 1;
 	private.startptr = InvalidXLogRecPtr;
@@ -1289,7 +1113,7 @@ main(int argc, char **argv)
 		{
 			if (config.stats == true)
 			{
-				XLogDumpCountRecord(&config, &stats, xlogreader_state);
+				XLogRecStoreStats(&stats, xlogreader_state);
 				stats.endptr = xlogreader_state->EndRecPtr;
 			}
 			else
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index 0e94833129..d7c35c37c4 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -329,6 +329,11 @@ extern XLogRecPtr RequestXLogSwitch(bool mark_unimportant);
 
 extern void GetOldestRestartPoint(XLogRecPtr *oldrecptr, TimeLineID *oldtli);
 
+extern void XLogRecGetBlockRefInfo(XLogReaderState *record,
+								   char *delimiter, uint32 *fpi_len,
+								   bool detailed_format,
+								   StringInfo blk_ref);
+
 /*
  * Exported for the functions in timeline.c and xlogarchive.c.  Only valid
  * in the startup process.
diff --git a/src/include/access/xlogstats.h b/src/include/access/xlogstats.h
new file mode 100644
index 0000000000..36d833f82b
--- /dev/null
+++ b/src/include/access/xlogstats.h
@@ -0,0 +1,40 @@
+/*-------------------------------------------------------------------------
+ *
+ * xlogstats.h
+ *		Definitions for WAL Statitstics
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		src/include/access/xlogstats.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef XLOGSTATS_H
+#define XLOGSTATS_H
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64	count;
+	uint64	rec_len;
+	uint64	fpi_len;
+} XLogRecStats;
+
+typedef struct XLogStats
+{
+	uint64	count;
+#ifdef FRONTEND
+	XLogRecPtr	startptr;
+	XLogRecPtr	endptr;
+#endif
+	XLogRecStats    rmgr_stats[RM_NEXT_ID];
+	XLogRecStats    record_stats[RM_NEXT_ID][MAX_XLINFO_TYPES];
+} XLogStats;
+
+extern void XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+						  uint32 *fpi_len);
+extern void XLogRecStoreStats(XLogStats *stats, XLogReaderState *record);
+
+#endif							/* XLOGSTATS_H */
-- 
2.25.1

v17-0002-pg_walinspect.patchapplication/octet-stream; name=v17-0002-pg_walinspect.patchDownload
From 064f7f5c304558510b0f1769a6114df6eb98e55d Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 6 Apr 2022 07:56:10 +0000
Subject: [PATCH v17] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 118 ++++
 contrib/pg_walinspect/pg_walinspect.c        | 639 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  13 +-
 src/backend/access/transam/xlogutils.c       |  33 +
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/access/xlogutils.h               |   4 +
 13 files changed, 843 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 332b486ecc..bbf220407b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -41,6 +41,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..aae6456c18
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,118 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_type text,
+    OUT record_length int4,
+    OUT main_data_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_type text,
+    OUT record_length int4,
+    OUT main_data_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_type text,
+    OUT record_length int4,
+    OUT main_data_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN  per_record boolean DEFAULT false,
+    OUT "resource_manager/record_type" text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    IN  per_record boolean DEFAULT false,
+    OUT "resource_manager/record_type" text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn, boolean) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn, boolean) TO pg_read_server_files;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..b93e16752c
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,639 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogstats.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats_till_end_of_wal);
+
+/*
+ * Struct holding information about the parameters that can be passed to
+ * GetWalStats.
+ */
+typedef struct GetWalStatsParams
+{
+	/* If true, generate statistics per-record instead of per-rmgr. */
+	bool stats_per_record;
+} GetWalStatsParams;
+
+typedef void (*GetWALDetailsCB) (FunctionCallInfo fcinfo,
+								 XLogRecPtr start_lsn,
+								 XLogRecPtr end_lsn,
+								 void *params);
+
+static bool IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetWALRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+							  GetWALDetailsCB wal_details_cb);
+static void GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+							  XLogRecPtr end_lsn, void *params);
+static void GetXLogSummaryStats(XLogStats * stats, ReturnSetInfo *rsinfo,
+								Datum *values, bool *nulls, uint32 ncols,
+								bool stats_per_record);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						XLogRecPtr end_lsn, void *params);
+
+/*
+ * Check if the given LSN is in future. Also, return the LSN up to which the
+ * server has WAL.
+ */
+static bool
+IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn)
+{
+	/*
+	 * We determine the current LSN of the server similar to how page_read
+	 * callback read_local_xlog_page_no_wait does.
+	 */
+	if (!RecoveryInProgress())
+		*curr_lsn = GetFlushRecPtr(NULL);
+	else
+		*curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(*curr_lsn));
+
+	if (lsn >= *curr_lsn)
+		return true;
+
+	return false;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page_no_wait,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get a single WAL record info.
+ */
+static void
+GetWALRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	const RmgrData *desc;
+	uint32	fpi_len = 0;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = &RmgrTable[XLogRecGetRmid(record)];
+	id = desc->rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		id = psprintf("UNKNOWN (%x)", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+
+	initStringInfo(&rec_desc);
+	desc->rm_desc(&rec_desc, record);
+
+	/* Block references. */
+	initStringInfo(&rec_blk_ref);
+	XLogRecGetBlockRefInfo(record, NULL, &fpi_len, true, &rec_blk_ref);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc->rm_name);
+	values[i++] = CStringGetTextDatum(id);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record info.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	Datum	result;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+	XLogRecPtr	lsn;
+	XLogRecPtr	curr_lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	HeapTuple	tuple;
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (IsFutureLSN(lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetWALRecordInfo(xlogreader, first_record, values, nulls,
+					 PG_GET_WAL_RECORD_INFO_COLS);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get WAL details such as record info, stats using the passed in callback.
+ */
+static void
+GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+				  GetWALDetailsCB wal_details_cb)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecPtr	curr_lsn;
+	GetWalStatsParams stats_params;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* If not till end of wal, end_lsn would have been specified. */
+	if (!till_end_of_wal)
+		end_lsn = PG_GETARG_LSN(1);
+
+	if (IsFutureLSN(start_lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	if (!till_end_of_wal && end_lsn >= curr_lsn)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+	else if (till_end_of_wal)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up to
+		 * "end".
+		 */
+		end_lsn = curr_lsn - 1;
+	}
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	if (wal_details_cb == GetWalStats)
+	{
+		MemSet(&stats_params, 0, sizeof(GetWalStatsParams));
+
+		if (!till_end_of_wal)
+			stats_params.stats_per_record = PG_GETARG_BOOL(2);
+		else
+			stats_params.stats_per_record = PG_GETARG_BOOL(1);
+
+		wal_details_cb(fcinfo, start_lsn, end_lsn, (void *) &stats_params);
+	}
+	else if (wal_details_cb == GetWALRecordsInfo)
+	{
+		wal_details_cb(fcinfo, start_lsn, end_lsn, NULL);
+	}
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+				  XLogRecPtr end_lsn, void *params)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+		{
+			GetWALRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+							 PG_GET_WAL_RECORDS_INFO_COLS);
+
+			tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+								 values, nulls);
+		}
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get info and data of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double	n_pct,
+			rec_len_pct,
+			fpi_len_pct,
+			tot_len_pct;
+	int	i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogStats *stats, ReturnSetInfo *rsinfo,
+					Datum *values, bool *nulls, uint32 ncols,
+					bool stats_per_record)
+{
+	uint64	total_count = 0;
+	uint64	total_rec_len = 0;
+	uint64	total_fpi_len = 0;
+	uint64	total_len = 0;
+	int	ri;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri < RM_NEXT_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		const RmgrData *desc = &RmgrTable[ri];
+
+		if (stats_per_record)
+		{
+			int rj;
+
+			for (rj = 0; rj < MAX_XLINFO_TYPES; rj++)
+			{
+				const char *id;
+
+				count = stats->record_stats[ri][rj].count;
+				rec_len = stats->record_stats[ri][rj].rec_len;
+				fpi_len = stats->record_stats[ri][rj].fpi_len;
+				tot_len = rec_len + fpi_len;
+
+				/* Skip undefined combinations and ones that didn't occur */
+				if (count == 0)
+					continue;
+
+				/* the upper four bits in xl_info are the rmgr's */
+				id = desc->rm_identify(rj << 4);
+				if (id == NULL)
+					id = psprintf("UNKNOWN (%x)", rj << 4);
+
+				FillXLogStatsRow(psprintf("%s/%s", desc->rm_name, id), count,
+								 total_count, rec_len, total_rec_len, fpi_len,
+								 total_fpi_len, tot_len, total_len,
+								 values, nulls, ncols);
+
+				tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+									 values, nulls);
+			}
+		}
+		else
+		{
+			count = stats->rmgr_stats[ri].count;
+			rec_len = stats->rmgr_stats[ri].rec_len;
+			fpi_len = stats->rmgr_stats[ri].fpi_len;
+			tot_len = rec_len + fpi_len;
+
+			FillXLogStatsRow(desc->rm_name, count, total_count, rec_len,
+							 total_rec_len, fpi_len, total_fpi_len, tot_len,
+							 total_len, values, nulls, ncols);
+
+			tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+								 values, nulls);
+		}
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+			XLogRecPtr end_lsn, void *params)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogStats stats;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_STATS_COLS];
+	bool	nulls[PG_GET_WAL_STATS_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+			XLogRecStoreStats(&stats, xlogreader);
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, rsinfo, values, nulls,
+						PG_GET_WAL_STATS_COLS,
+						((GetWalStatsParams *)params)->stats_per_record);
+
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get stats of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWalStats);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get stats of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWalStats);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index e437c42992..585c94c488 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -1320,13 +1320,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1447,6 +1440,12 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index a4dedc58b7..50159fd4cc 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -79,6 +79,10 @@ typedef struct xl_invalid_page
 
 static HTAB *invalid_page_tab = NULL;
 
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal);
 
 /* Report a reference to an invalid page */
 static void
@@ -851,6 +855,31 @@ wal_segment_close(XLogReaderState *state)
 int
 read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 					 int reqLen, XLogRecPtr targetRecPtr, char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, true);
+}
+
+/*
+ * Same as read_local_xlog_page except that it doesn't wait for future WAL
+ * to be available.
+ */
+int
+read_local_xlog_page_no_wait(XLogReaderState *state, XLogRecPtr targetPagePtr,
+							 int reqLen, XLogRecPtr targetRecPtr,
+							 char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, false);
+}
+
+/*
+ * Implementation of read_local_xlog_page and its no wait version.
+ */
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal)
 {
 	XLogRecPtr	read_upto,
 				loc;
@@ -906,6 +935,10 @@ read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 			if (loc <= read_upto)
 				break;
 
+			/* If asked, let's not wait for future WAL. */
+			if (!wait_for_wal)
+				break;
+
 			CHECK_FOR_INTERRUPTS();
 			pg_usleep(1000L);
 		}
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 61f06e828f..9dd9f05204 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -27,6 +27,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 09f6464331..3e644372f9 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index d7c35c37c4..2985c75361 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -319,7 +319,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern const RmgrData RmgrTable[];
+extern PGDLLIMPORT const RmgrData RmgrTable[];
 
 /*
  * Exported to support xlog switching from checkpointer
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index f4388cc9be..b4c7d93787 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -340,9 +340,7 @@ extern void XLogReaderSetDecodeBuffer(XLogReaderState *state,
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Return values from XLogPageReadCB. */
 typedef enum XLogPageReadResult
diff --git a/src/include/access/xlogutils.h b/src/include/access/xlogutils.h
index 64708949db..22c2299d68 100644
--- a/src/include/access/xlogutils.h
+++ b/src/include/access/xlogutils.h
@@ -92,6 +92,10 @@ extern void FreeFakeRelcacheEntry(Relation fakerel);
 extern int	read_local_xlog_page(XLogReaderState *state,
 								 XLogRecPtr targetPagePtr, int reqLen,
 								 XLogRecPtr targetRecPtr, char *cur_page);
+extern int	read_local_xlog_page_no_wait(XLogReaderState *state,
+										 XLogRecPtr targetPagePtr, int reqLen,
+										 XLogRecPtr targetRecPtr,
+										 char *cur_page);
 extern void wal_segment_open(XLogReaderState *state,
 							 XLogSegNo nextSegNo,
 							 TimeLineID *tli_p);
-- 
2.25.1

v17-0003-pg_walinspect-tests.patchapplication/octet-stream; name=v17-0003-pg_walinspect-tests.patchDownload
From 66cf3eca33d413cdfa59a4c5a5d4e8b164c7d3a1 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 6 Apr 2022 06:25:48 +0000
Subject: [PATCH v17] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 145 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 107 +++++++++++++
 2 files changed, 252 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..53d3146fe1
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,145 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+-- Functions accessible by users with role pg_read_server_files
+GRANT pg_read_server_files TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..6e120c472b
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,107 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- no
+
+-- Functions accessible by users with role pg_read_server_files
+
+GRANT pg_read_server_files TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- yes
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v17-0004-pg_walinspect-docs.patchapplication/octet-stream; name=v17-0004-pg_walinspect-docs.patchDownload
From 766f3f862fbcc19fc6e961f82dca59ab8386149b Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Wed, 6 Apr 2022 08:00:06 +0000
Subject: [PATCH v17] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 244 +++++++++++++++++++++++++++++++++
 3 files changed, 246 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index 1e42ce1a7f..4e7b87a42f 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -131,6 +131,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index fd853af01f..34c19c80f1 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -147,6 +147,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..e344951f0d
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,244 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_read_server_files</literal> role. Access may be granted by
+  superusers to others using <command>GRANT</command>.
+ </para>
+    
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_type OUT text,
+                             record_length OUT int4,
+                             main_data_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of a given LSN. If the given LSN isn't
+      containing a valid WAL record, it gives the information of the next
+      available valid WAL record. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_type OUT text,
+                              record_length OUT int4,
+                              main_data_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_type, record_length, main_data_length, fpi_length, description from pg_get_wal_records_info('0/14FBA30', '0/15011D7');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager |  record_type  | record_length | main_data_length | fpi_length |                       description                       
+-----------+-----------+-----------+-----+------------------+---------------+---------------+------------------+------------+---------------------------------------------------------
+ 0/14FBA30 | 0/14FBA67 | 0/14FB9F8 |   0 | Heap2            | PRUNE         |            56 |                8 |          0 | latestRemovedXid 0 nredirected 0 ndead 1
+ 0/14FBA68 | 0/14FBA9F | 0/14FBA30 |   0 | Standby          | RUNNING_XACTS |            50 |               24 |          0 | nextXid 723 latestCompletedXid 722 oldestRunningXid 723
+ 0/14FBAA0 | 0/14FBACF | 0/14FBA68 |   0 | Storage          | CREATE        |            42 |               16 |          0 | base/5/16390
+ 0/14FBAD0 | 0/14FC117 | 0/14FBAA0 | 723 | Heap             | INSERT        |          1582 |                3 |       1528 | off 8 flags 0x01
+ 0/14FC118 | 0/14FD487 | 0/14FBAD0 | 723 | Btree            | INSERT_LEAF   |          4973 |                2 |       4920 | off 244
+ 0/14FD488 | 0/14FEFCF | 0/14FC118 | 723 | Btree            | INSERT_LEAF   |          6953 |                2 |       6900 | off 126
+ 0/14FEFD0 | 0/14FF027 | 0/14FD488 | 723 | Heap2            | MULTI_INSERT  |            85 |                6 |          0 | 1 tuples flags 0x02
+ 0/14FF028 | 0/14FF06F | 0/14FEFD0 | 723 | Btree            | INSERT_LEAF   |            72 |                2 |          0 | off 155
+ 0/14FF070 | 0/14FF0B7 | 0/14FF028 | 723 | Btree            | INSERT_LEAF   |            72 |                2 |          0 | off 134
+ 0/14FF0B8 | 0/14FF18F | 0/14FF070 | 723 | Heap             | INSERT        |           211 |                3 |          0 | off 9 flags 0x00
+ 0/14FF190 | 0/14FF1CF | 0/14FF0B8 | 723 | Btree            | INSERT_LEAF   |            64 |                2 |          0 | off 244
+ 0/14FF1D0 | 0/15010B7 | 0/14FF190 | 723 | Btree            | SPLIT_L       |          7885 |               10 |       4136 | level 0, firstrightoff 120, newitemoff 47, postingoff 0
+ 0/15010B8 | 0/150117F | 0/14FF1D0 | 723 | Btree            | INSERT_UPPER  |           197 |                2 |        136 | off 2
+ 0/1501180 | 0/15011D7 | 0/15010B8 | 723 | Heap2            | MULTI_INSERT  |            85 |                6 |          0 | 1 tuples flags 0x02
+(14 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info_till_end_of_wal(start_lsn pg_lsn,
+                                              start_lsn OUT pg_lsn,
+                                              end_lsn OUT pg_lsn,
+                                              prev_lsn OUT pg_lsn,
+                                              xid OUT xid,
+                                              resource_manager OUT text,
+                                              record_type OUT text,
+                                              record_length OUT int4,
+                                              main_data_length OUT int4,
+                                              fpi_length OUT int4,
+                                              description OUT text,
+                                              block_ref OUT text)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_records_info()</function>
+      except that it gets information of all the valid WAL records from 
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       per_record boolean DEFAULT false,
+                       "resource_manager/record_type" OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4,
+                       combined_size OUT int8,
+                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and
+      <replaceable>end_lsn</replaceable>. By default, it returns one row per
+      <replaceable>resource_manager</replaceable> type. When
+      <replaceable>per_record</replaceable> is set to <literal>true</literal>,
+      it returns one row per <replaceable>record_type</replaceable>. This
+      function emits an error if a future (the LSN database system doesn't know
+      about) <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/12FBA30', '0/15011D7') where count > 0;
+ resource_manager/record_type | count | count_percentage | record_size | record_size_percentage | fpi_size | fpi_size_percentage | combined_size | combined_size_percentage 
+------------------------------+-------+------------------+-------------+------------------------+----------+---------------------+---------------+--------------------------
+ XLOG                         |    12 |       0.13002492 |        1024 |          2.3833392e-05 |      352 |          0.06136488 |          1376 |            3.2021846e-05
+ Transaction                  |   188 |        2.0370572 |       62903 |           0.0014640546 |        0 |                   0 |         62903 |             0.0014638591
+ Storage                      |    13 |       0.14086033 |         546 |          1.2708038e-05 |        0 |                   0 |           546 |            1.2706342e-05
+ Database                     |     2 |       0.02167082 |          84 |           1.955083e-06 |        0 |                   0 |            84 |             1.954822e-06
+ Standby                      |   219 |        2.3729548 |       15830 |          0.00036844003 |        0 |                   0 |         15830 |            0.00036839084
+ Heap2                        |  1905 |        20.641457 |      384619 |           0.0089519285 |   364472 |            63.53915 |        749091 |              0.017432613
+ Heap                         |  1319 |        14.291906 |      621997 |            0.014476853 |   145232 |           25.318592 |        767229 |              0.017854715
+ Btree                        |  5571 |         60.36407 |  4295405999 |                99.9747 |    63562 |          11.0808935 |    4295469561 |                 99.96284
+(8 rows)
+</screen>
+
+With <replaceable>per_record</replaceable> passed as <literal>true</literal>:
+
+<screen>
+postgres=# select * from pg_get_wal_stats('0/14FBA30', '0/15011D7', true) where count > 0;
+ resource_manager/record_type | count | count_percentage | record_size | record_size_percentage | fpi_size | fpi_size_percentage | combined_size | combined_size_percentage 
+------------------------------+-------+------------------+-------------+------------------------+----------+---------------------+---------------+--------------------------
+ Storage/CREATE               |     1 |         7.142857 |          42 |              0.8922881 |        0 |                   0 |            42 |               0.18811305
+ Standby/RUNNING_XACTS        |     1 |         7.142857 |          50 |              1.0622478 |        0 |                   0 |            50 |                0.2239441
+ Heap2/PRUNE                  |     1 |         7.142857 |          56 |              1.1897174 |        0 |                   0 |            56 |                0.2508174
+ Heap2/MULTI_INSERT           |     2 |        14.285714 |         170 |              3.6116421 |        0 |                   0 |           170 |               0.76140994
+ Heap/INSERT                  |     2 |        14.285714 |         265 |               5.629913 |     1528 |            8.671964 |          1793 |                 8.030636
+ Btree/INSERT_LEAF            |     5 |        35.714287 |         314 |              6.6709156 |    11820 |            67.08286 |         12134 |                54.346756
+ Btree/INSERT_UPPER           |     1 |         7.142857 |          61 |              1.2959422 |      136 |          0.77185017 |           197 |                0.8823398
+ Btree/SPLIT_L                |     1 |         7.142857 |        3749 |               79.64733 |     4136 |           23.473326 |          7885 |                35.315987
+(8 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats_till_end_of_wal(start_lsn pg_lsn,
+                                       per_record boolean DEFAULT false,
+                                       "resource_manager/record_type" OUT text,
+                                       count OUT int8,
+                                       count_percentage OUT float4,
+                                       record_length OUT int8,
+                                       record_length_percentage OUT float4,
+                                       fpi_length OUT int8,
+                                       fpi_length_percentage OUT float4,
+                                       combined_size OUT int8,
+                                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_stats()</function> except
+      that it gets statistics of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#87Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Bharath Rupireddy (#86)
4 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Wed, Apr 6, 2022 at 2:15 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

Attaching v17 patch-set with the above review comments addressed.
Please have a look at it.

Had to rebase because of 5c279a6d350 (Custom WAL Resource Managers.).
Please find v18 patch-set.

Regards,
Bharath Rupireddy.

Attachments:

v18-0001-Refactor-pg_waldump-code.patchapplication/x-patch; name=v18-0001-Refactor-pg_waldump-code.patchDownload
From eb9e3464340a4e09236b9f50e596c9a1360b0886 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 7 Apr 2022 08:36:01 +0000
Subject: [PATCH v18] Refactor pg_waldump code

This patch puts some generic chunks of pg_waldump's code into
separate reusable functions in xlogdesc.c and xlogstats.c, a new
file along xlogstats.h introduced for placing WAL stats and
structures.

This way, other modules can reuse these common functions.
---
 src/backend/access/rmgrdesc/xlogdesc.c | 125 ++++++++++++++++
 src/backend/access/transam/Makefile    |   1 +
 src/backend/access/transam/xlogstats.c |  93 ++++++++++++
 src/bin/pg_waldump/.gitignore          |   1 +
 src/bin/pg_waldump/Makefile            |   8 +-
 src/bin/pg_waldump/pg_waldump.c        | 199 ++-----------------------
 src/include/access/xlog_internal.h     |   5 +
 src/include/access/xlogstats.h         |  40 +++++
 8 files changed, 282 insertions(+), 190 deletions(-)
 create mode 100644 src/backend/access/transam/xlogstats.c
 create mode 100644 src/include/access/xlogstats.h

diff --git a/src/backend/access/rmgrdesc/xlogdesc.c b/src/backend/access/rmgrdesc/xlogdesc.c
index e7452af679..429e5dcd5b 100644
--- a/src/backend/access/rmgrdesc/xlogdesc.c
+++ b/src/backend/access/rmgrdesc/xlogdesc.c
@@ -200,3 +200,128 @@ xlog_identify(uint8 info)
 
 	return id;
 }
+
+/*
+ * Returns a string giving information about all the blocks in an
+ * XLogRecord.
+ */
+void
+XLogRecGetBlockRefInfo(XLogReaderState *record, char *delimiter,
+					   uint32 *fpi_len, bool detailed_format,
+					   StringInfo buf)
+{
+	int	block_id;
+
+	Assert(record != NULL);
+
+	if (detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		RelFileNode rnode = {InvalidOid, InvalidOid, InvalidOid};
+		ForkNumber	forknum = InvalidForkNumber;
+		BlockNumber blk = InvalidBlockNumber;
+
+		if (!XLogRecHasBlockRef(record, block_id))
+			continue;
+
+		XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
+
+		if (detailed_format)
+		{
+			/* Get block references in detailed format. */
+
+			appendStringInfo(buf,
+							 "\tblkref #%d: rel %u/%u/%u fork %s blk %u",
+							 block_id,
+							 rnode.spcNode, rnode.dbNode, rnode.relNode,
+							 forkNames[forknum],
+							 blk);
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
+
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (BKPIMAGE_COMPRESSED(bimg_info))
+				{
+					const char *method;
+
+					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
+						method = "pglz";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
+						method = "lz4";
+					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
+						method = "zstd";
+					else
+						method = "unknown";
+
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u, "
+									 "compression saved: %u, method: %s",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length,
+									 BLCKSZ -
+									 XLogRecGetBlock(record, block_id)->hole_length -
+									 XLogRecGetBlock(record, block_id)->bimg_len,
+									 method);
+				}
+				else
+				{
+					appendStringInfo(buf,
+									 " (FPW%s); hole: offset: %u, length: %u",
+									 XLogRecBlockImageApply(record, block_id) ?
+									 "" : " for WAL verification",
+									 XLogRecGetBlock(record, block_id)->hole_offset,
+									 XLogRecGetBlock(record, block_id)->hole_length);
+				}
+			}
+		}
+		else
+		{
+			/* Get block references in short format. */
+
+			if (forknum != MAIN_FORKNUM)
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u fork %s blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 forkNames[forknum],
+								 blk);
+			}
+			else
+			{
+				appendStringInfo(buf,
+								 ", blkref #%d: rel %u/%u/%u blk %u",
+								 block_id,
+								 rnode.spcNode, rnode.dbNode, rnode.relNode,
+								 blk);
+			}
+
+			if (XLogRecHasBlockImage(record, block_id))
+			{
+				/* Calculate the amount of FPI data in the record. */
+				if (fpi_len)
+					*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+
+				if (XLogRecBlockImageApply(record, block_id))
+					appendStringInfo(buf, " FPW");
+				else
+					appendStringInfo(buf, " FPW for WAL verification");
+			}
+		}
+
+		if (detailed_format && delimiter != NULL)
+			appendStringInfoChar(buf, '\n');
+	}
+
+	if (!detailed_format && delimiter != NULL)
+		appendStringInfoChar(buf, '\n');
+}
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 8c17c88dfc..3e5444a6f7 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -34,6 +34,7 @@ OBJS = \
 	xlogprefetcher.o \
 	xlogreader.o \
 	xlogrecovery.o \
+	xlogstats.o \
 	xlogutils.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/transam/xlogstats.c b/src/backend/access/transam/xlogstats.c
new file mode 100644
index 0000000000..aff3069ecb
--- /dev/null
+++ b/src/backend/access/transam/xlogstats.c
@@ -0,0 +1,93 @@
+/*-------------------------------------------------------------------------
+ *
+ * xlogstats.c
+ *		Functions for WAL Statitstics
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		src/backend/access/transam/xlogstats.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlogreader.h"
+#include "access/xlogstats.h"
+
+/*
+ * Calculate the size of a record, split into !FPI and FPI parts.
+ */
+void
+XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+			  uint32 *fpi_len)
+{
+	int	block_id;
+
+	/*
+	 * Calculate the amount of FPI data in the record.
+	 *
+	 * XXX: We peek into xlogreader's private decoded backup blocks for the
+	 * bimg_len indicating the length of FPI data.
+	 */
+	*fpi_len = 0;
+	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
+	{
+		if (XLogRecHasBlockImage(record, block_id))
+			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
+	}
+
+	/*
+	 * Calculate the length of the record as the total length - the length of
+	 * all the block images.
+	 */
+	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
+}
+
+/*
+ * Store per-rmgr and per-record statistics for a given record.
+ */
+void
+XLogRecStoreStats(XLogStats *stats, XLogReaderState *record)
+{
+	RmgrId	rmid;
+	uint8	recid;
+	uint32	rec_len;
+	uint32	fpi_len;
+
+	Assert(stats != NULL && record != NULL);
+
+	stats->count++;
+
+	rmid = XLogRecGetRmid(record);
+
+	XLogRecGetLen(record, &rec_len, &fpi_len);
+
+	/* Update per-rmgr statistics */
+
+	stats->rmgr_stats[rmid].count++;
+	stats->rmgr_stats[rmid].rec_len += rec_len;
+	stats->rmgr_stats[rmid].fpi_len += fpi_len;
+
+	/*
+	 * Update per-record statistics, where the record is identified by a
+	 * combination of the RmgrId and the four bits of the xl_info field that
+	 * are the rmgr's domain (resulting in sixteen possible entries per
+	 * RmgrId).
+	 */
+
+	recid = XLogRecGetInfo(record) >> 4;
+
+	/*
+	 * XACT records need to be handled differently. Those records use the
+	 * first bit of those four bits for an optional flag variable and the
+	 * following three bits for the opcode. We filter opcode out of xl_info
+	 * and use it as the identifier of the record.
+	 */
+	if (rmid == RM_XACT_ID)
+		recid &= 0x07;
+
+	stats->record_stats[rmid][recid].count++;
+	stats->record_stats[rmid][recid].rec_len += rec_len;
+	stats->record_stats[rmid][recid].fpi_len += fpi_len;
+}
diff --git a/src/bin/pg_waldump/.gitignore b/src/bin/pg_waldump/.gitignore
index 3be00a8b61..dabb6e34b6 100644
--- a/src/bin/pg_waldump/.gitignore
+++ b/src/bin/pg_waldump/.gitignore
@@ -23,6 +23,7 @@
 /xactdesc.c
 /xlogdesc.c
 /xlogreader.c
+/xlogstat.c
 
 # Generated by test suite
 /tmp_check/
diff --git a/src/bin/pg_waldump/Makefile b/src/bin/pg_waldump/Makefile
index 9f333d0c8a..d6459e17c7 100644
--- a/src/bin/pg_waldump/Makefile
+++ b/src/bin/pg_waldump/Makefile
@@ -13,7 +13,8 @@ OBJS = \
 	compat.o \
 	pg_waldump.o \
 	rmgrdesc.o \
-	xlogreader.o
+	xlogreader.o \
+	xlogstats.o
 
 override CPPFLAGS := -DFRONTEND $(CPPFLAGS)
 
@@ -29,6 +30,9 @@ pg_waldump: $(OBJS) | submake-libpgport
 xlogreader.c: % : $(top_srcdir)/src/backend/access/transam/%
 	rm -f $@ && $(LN_S) $< .
 
+xlogstats.c: % : $(top_srcdir)/src/backend/access/transam/%
+	rm -f $@ && $(LN_S) $< .
+
 $(RMGRDESCSOURCES): % : $(top_srcdir)/src/backend/access/rmgrdesc/%
 	rm -f $@ && $(LN_S) $< .
 
@@ -42,7 +46,7 @@ uninstall:
 	rm -f '$(DESTDIR)$(bindir)/pg_waldump$(X)'
 
 clean distclean maintainer-clean:
-	rm -f pg_waldump$(X) $(OBJS) $(RMGRDESCSOURCES) xlogreader.c
+	rm -f pg_waldump$(X) $(OBJS) $(RMGRDESCSOURCES) xlogreader.c xlogstats.c
 	rm -rf tmp_check
 
 check:
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index 30ca7684bd..a0ab1912f4 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -21,6 +21,7 @@
 #include "access/xlog_internal.h"
 #include "access/xlogreader.h"
 #include "access/xlogrecord.h"
+#include "access/xlogstats.h"
 #include "common/fe_memutils.h"
 #include "common/logging.h"
 #include "getopt_long.h"
@@ -66,24 +67,6 @@ typedef struct XLogDumpConfig
 	bool		filter_by_fpw;
 } XLogDumpConfig;
 
-typedef struct Stats
-{
-	uint64		count;
-	uint64		rec_len;
-	uint64		fpi_len;
-} Stats;
-
-#define MAX_XLINFO_TYPES 16
-
-typedef struct XLogDumpStats
-{
-	uint64		count;
-	XLogRecPtr	startptr;
-	XLogRecPtr	endptr;
-	Stats		rmgr_stats[RM_MAX_ID + 1];
-	Stats		record_stats[RM_MAX_ID + 1][MAX_XLINFO_TYPES];
-} XLogDumpStats;
-
 #define fatal_error(...) do { pg_log_fatal(__VA_ARGS__); exit(EXIT_FAILURE); } while(0)
 
 /*
@@ -453,81 +436,6 @@ XLogRecordHasFPW(XLogReaderState *record)
 	return false;
 }
 
-/*
- * Calculate the size of a record, split into !FPI and FPI parts.
- */
-static void
-XLogDumpRecordLen(XLogReaderState *record, uint32 *rec_len, uint32 *fpi_len)
-{
-	int			block_id;
-
-	/*
-	 * Calculate the amount of FPI data in the record.
-	 *
-	 * XXX: We peek into xlogreader's private decoded backup blocks for the
-	 * bimg_len indicating the length of FPI data.
-	 */
-	*fpi_len = 0;
-	for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-	{
-		if (XLogRecHasBlockImage(record, block_id))
-			*fpi_len += XLogRecGetBlock(record, block_id)->bimg_len;
-	}
-
-	/*
-	 * Calculate the length of the record as the total length - the length of
-	 * all the block images.
-	 */
-	*rec_len = XLogRecGetTotalLen(record) - *fpi_len;
-}
-
-/*
- * Store per-rmgr and per-record statistics for a given record.
- */
-static void
-XLogDumpCountRecord(XLogDumpConfig *config, XLogDumpStats *stats,
-					XLogReaderState *record)
-{
-	RmgrId		rmid;
-	uint8		recid;
-	uint32		rec_len;
-	uint32		fpi_len;
-
-	stats->count++;
-
-	rmid = XLogRecGetRmid(record);
-
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
-
-	/* Update per-rmgr statistics */
-
-	stats->rmgr_stats[rmid].count++;
-	stats->rmgr_stats[rmid].rec_len += rec_len;
-	stats->rmgr_stats[rmid].fpi_len += fpi_len;
-
-	/*
-	 * Update per-record statistics, where the record is identified by a
-	 * combination of the RmgrId and the four bits of the xl_info field that
-	 * are the rmgr's domain (resulting in sixteen possible entries per
-	 * RmgrId).
-	 */
-
-	recid = XLogRecGetInfo(record) >> 4;
-
-	/*
-	 * XACT records need to be handled differently. Those records use the
-	 * first bit of those four bits for an optional flag variable and the
-	 * following three bits for the opcode. We filter opcode out of xl_info
-	 * and use it as the identifier of the record.
-	 */
-	if (rmid == RM_XACT_ID)
-		recid &= 0x07;
-
-	stats->record_stats[rmid][recid].count++;
-	stats->record_stats[rmid][recid].rec_len += rec_len;
-	stats->record_stats[rmid][recid].fpi_len += fpi_len;
-}
-
 /*
  * Print a record to stdout
  */
@@ -538,15 +446,12 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	const RmgrDescData *desc = GetRmgrDesc(XLogRecGetRmid(record));
 	uint32		rec_len;
 	uint32		fpi_len;
-	RelFileNode rnode;
-	ForkNumber	forknum;
-	BlockNumber blk;
-	int			block_id;
 	uint8		info = XLogRecGetInfo(record);
 	XLogRecPtr	xl_prev = XLogRecGetPrev(record);
 	StringInfoData s;
+	char	delim = {'\n'};
 
-	XLogDumpRecordLen(record, &rec_len, &fpi_len);
+	XLogRecGetLen(record, &rec_len, &fpi_len);
 
 	printf("rmgr: %-11s len (rec/tot): %6u/%6u, tx: %10u, lsn: %X/%08X, prev %X/%08X, ",
 		   desc->rm_name,
@@ -564,93 +469,11 @@ XLogDumpDisplayRecord(XLogDumpConfig *config, XLogReaderState *record)
 	initStringInfo(&s);
 	desc->rm_desc(&s, record);
 	printf("%s", s.data);
-	pfree(s.data);
 
-	if (!config->bkp_details)
-	{
-		/* print block references (short format) */
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			if (forknum != MAIN_FORKNUM)
-				printf(", blkref #%d: rel %u/%u/%u fork %s blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   forkNames[forknum],
-					   blk);
-			else
-				printf(", blkref #%d: rel %u/%u/%u blk %u",
-					   block_id,
-					   rnode.spcNode, rnode.dbNode, rnode.relNode,
-					   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				if (XLogRecBlockImageApply(record, block_id))
-					printf(" FPW");
-				else
-					printf(" FPW for WAL verification");
-			}
-		}
-		putchar('\n');
-	}
-	else
-	{
-		/* print block references (detailed format) */
-		putchar('\n');
-		for (block_id = 0; block_id <= XLogRecMaxBlockId(record); block_id++)
-		{
-			if (!XLogRecHasBlockRef(record, block_id))
-				continue;
-
-			XLogRecGetBlockTag(record, block_id, &rnode, &forknum, &blk);
-			printf("\tblkref #%d: rel %u/%u/%u fork %s blk %u",
-				   block_id,
-				   rnode.spcNode, rnode.dbNode, rnode.relNode,
-				   forkNames[forknum],
-				   blk);
-			if (XLogRecHasBlockImage(record, block_id))
-			{
-				uint8		bimg_info = XLogRecGetBlock(record, block_id)->bimg_info;
-
-				if (BKPIMAGE_COMPRESSED(bimg_info))
-				{
-					const char *method;
-
-					if ((bimg_info & BKPIMAGE_COMPRESS_PGLZ) != 0)
-						method = "pglz";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_LZ4) != 0)
-						method = "lz4";
-					else if ((bimg_info & BKPIMAGE_COMPRESS_ZSTD) != 0)
-						method = "zstd";
-					else
-						method = "unknown";
-
-					printf(" (FPW%s); hole: offset: %u, length: %u, "
-						   "compression saved: %u, method: %s",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length,
-						   BLCKSZ -
-						   XLogRecGetBlock(record, block_id)->hole_length -
-						   XLogRecGetBlock(record, block_id)->bimg_len,
-						   method);
-				}
-				else
-				{
-					printf(" (FPW%s); hole: offset: %u, length: %u",
-						   XLogRecBlockImageApply(record, block_id) ?
-						   "" : " for WAL verification",
-						   XLogRecGetBlock(record, block_id)->hole_offset,
-						   XLogRecGetBlock(record, block_id)->hole_length);
-				}
-			}
-			putchar('\n');
-		}
-	}
+	resetStringInfo(&s);
+	XLogRecGetBlockRefInfo(record, &delim, NULL, config->bkp_details, &s);
+	printf("%s", s.data);
+	pfree(s.data);
 }
 
 /*
@@ -698,7 +521,7 @@ XLogDumpStatsRow(const char *name,
  * Display summary statistics about the records seen so far.
  */
 static void
-XLogDumpDisplayStats(XLogDumpConfig *config, XLogDumpStats *stats)
+XLogDumpDisplayStats(XLogDumpConfig *config, XLogStats *stats)
 {
 	int			ri,
 				rj;
@@ -867,7 +690,7 @@ main(int argc, char **argv)
 	XLogReaderState *xlogreader_state;
 	XLogDumpPrivate private;
 	XLogDumpConfig config;
-	XLogDumpStats stats;
+	XLogStats stats;
 	XLogRecord *record;
 	XLogRecPtr	first_record;
 	char	   *waldir = NULL;
@@ -921,7 +744,7 @@ main(int argc, char **argv)
 
 	memset(&private, 0, sizeof(XLogDumpPrivate));
 	memset(&config, 0, sizeof(XLogDumpConfig));
-	memset(&stats, 0, sizeof(XLogDumpStats));
+	memset(&stats, 0, sizeof(XLogStats));
 
 	private.timeline = 1;
 	private.startptr = InvalidXLogRecPtr;
@@ -1319,7 +1142,7 @@ main(int argc, char **argv)
 		{
 			if (config.stats == true)
 			{
-				XLogDumpCountRecord(&config, &stats, xlogreader_state);
+				XLogRecStoreStats(&stats, xlogreader_state);
 				stats.endptr = xlogreader_state->EndRecPtr;
 			}
 			else
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index f69ea2355d..d0206f7c74 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -348,6 +348,11 @@ extern XLogRecPtr RequestXLogSwitch(bool mark_unimportant);
 
 extern void GetOldestRestartPoint(XLogRecPtr *oldrecptr, TimeLineID *oldtli);
 
+extern void XLogRecGetBlockRefInfo(XLogReaderState *record,
+								   char *delimiter, uint32 *fpi_len,
+								   bool detailed_format,
+								   StringInfo blk_ref);
+
 /*
  * Exported for the functions in timeline.c and xlogarchive.c.  Only valid
  * in the startup process.
diff --git a/src/include/access/xlogstats.h b/src/include/access/xlogstats.h
new file mode 100644
index 0000000000..227d59ce17
--- /dev/null
+++ b/src/include/access/xlogstats.h
@@ -0,0 +1,40 @@
+/*-------------------------------------------------------------------------
+ *
+ * xlogstats.h
+ *		Definitions for WAL Statitstics
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		src/include/access/xlogstats.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef XLOGSTATS_H
+#define XLOGSTATS_H
+
+#define MAX_XLINFO_TYPES 16
+
+typedef struct XLogRecStats
+{
+	uint64	count;
+	uint64	rec_len;
+	uint64	fpi_len;
+} XLogRecStats;
+
+typedef struct XLogStats
+{
+	uint64	count;
+#ifdef FRONTEND
+	XLogRecPtr	startptr;
+	XLogRecPtr	endptr;
+#endif
+	XLogRecStats    rmgr_stats[RM_MAX_ID  + 1];
+	XLogRecStats    record_stats[RM_MAX_ID  + 1][MAX_XLINFO_TYPES];
+} XLogStats;
+
+extern void XLogRecGetLen(XLogReaderState *record, uint32 *rec_len,
+						  uint32 *fpi_len);
+extern void XLogRecStoreStats(XLogStats *stats, XLogReaderState *record);
+
+#endif							/* XLOGSTATS_H */
-- 
2.25.1

v18-0002-pg_walinspect.patchapplication/x-patch; name=v18-0002-pg_walinspect.patchDownload
From 827c8b6f4300acb99b779cec41ecc0411897bda8 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 7 Apr 2022 09:15:59 +0000
Subject: [PATCH v18] pg_walinspect

---
 contrib/Makefile                             |   1 +
 contrib/pg_walinspect/.gitignore             |   4 +
 contrib/pg_walinspect/Makefile               |  26 +
 contrib/pg_walinspect/pg_walinspect--1.0.sql | 118 ++++
 contrib/pg_walinspect/pg_walinspect.c        | 650 +++++++++++++++++++
 contrib/pg_walinspect/pg_walinspect.control  |   5 +
 src/backend/access/transam/xlogreader.c      |  13 +-
 src/backend/access/transam/xlogutils.c       |  33 +
 src/bin/pg_waldump/pg_waldump.c              |   5 +
 src/include/access/xlog.h                    |   2 +-
 src/include/access/xlog_internal.h           |   2 +-
 src/include/access/xlogreader.h              |   2 -
 src/include/access/xlogutils.h               |   4 +
 13 files changed, 854 insertions(+), 11 deletions(-)
 create mode 100644 contrib/pg_walinspect/.gitignore
 create mode 100644 contrib/pg_walinspect/Makefile
 create mode 100644 contrib/pg_walinspect/pg_walinspect--1.0.sql
 create mode 100644 contrib/pg_walinspect/pg_walinspect.c
 create mode 100644 contrib/pg_walinspect/pg_walinspect.control

diff --git a/contrib/Makefile b/contrib/Makefile
index 332b486ecc..bbf220407b 100644
--- a/contrib/Makefile
+++ b/contrib/Makefile
@@ -41,6 +41,7 @@ SUBDIRS = \
 		pgrowlocks	\
 		pgstattuple	\
 		pg_visibility	\
+		pg_walinspect	\
 		postgres_fdw	\
 		seg		\
 		spi		\
diff --git a/contrib/pg_walinspect/.gitignore b/contrib/pg_walinspect/.gitignore
new file mode 100644
index 0000000000..5dcb3ff972
--- /dev/null
+++ b/contrib/pg_walinspect/.gitignore
@@ -0,0 +1,4 @@
+# Generated subdirectories
+/log/
+/results/
+/tmp_check/
diff --git a/contrib/pg_walinspect/Makefile b/contrib/pg_walinspect/Makefile
new file mode 100644
index 0000000000..c92a97447f
--- /dev/null
+++ b/contrib/pg_walinspect/Makefile
@@ -0,0 +1,26 @@
+# contrib/pg_walinspect/Makefile
+
+MODULE_big = pg_walinspect
+OBJS = \
+	$(WIN32RES) \
+	pg_walinspect.o
+PGFILEDESC = "pg_walinspect - functions to inspect contents of PostgreSQL Write-Ahead Log"
+
+PG_CPPFLAGS = -I$(libpq_srcdir)
+SHLIB_LINK_INTERNAL = $(libpq)
+
+EXTENSION = pg_walinspect
+DATA = pg_walinspect--1.0.sql
+
+REGRESS = pg_walinspect
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/pg_walinspect
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/pg_walinspect/pg_walinspect--1.0.sql b/contrib/pg_walinspect/pg_walinspect--1.0.sql
new file mode 100644
index 0000000000..aae6456c18
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect--1.0.sql
@@ -0,0 +1,118 @@
+/* contrib/pg_walinspect/pg_walinspect--1.0.sql */
+
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION pg_walinspect" to load this file. \quit
+
+--
+-- pg_get_wal_record_info()
+--
+CREATE FUNCTION pg_get_wal_record_info(IN in_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_type text,
+    OUT record_length int4,
+    OUT main_data_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text
+)
+AS 'MODULE_PATHNAME', 'pg_get_wal_record_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info()
+--
+CREATE FUNCTION pg_get_wal_records_info(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_type text,
+    OUT record_length int4,
+    OUT main_data_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_records_info_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_records_info_till_end_of_wal(IN start_lsn pg_lsn,
+    OUT start_lsn pg_lsn,
+    OUT end_lsn pg_lsn,
+    OUT prev_lsn pg_lsn,
+    OUT xid xid,
+    OUT resource_manager text,
+    OUT record_type text,
+    OUT record_length int4,
+    OUT main_data_length int4,
+    OUT fpi_length int4,
+	OUT description text,
+    OUT block_ref text
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_records_info_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info_till_end_of_wal(pg_lsn) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats()
+--
+CREATE FUNCTION pg_get_wal_stats(IN start_lsn pg_lsn,
+    IN end_lsn pg_lsn,
+    IN  per_record boolean DEFAULT false,
+    OUT "resource_manager/record_type" text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean) TO pg_read_server_files;
+
+--
+-- pg_get_wal_stats_till_end_of_wal()
+--
+CREATE FUNCTION pg_get_wal_stats_till_end_of_wal(IN start_lsn pg_lsn,
+    IN  per_record boolean DEFAULT false,
+    OUT "resource_manager/record_type" text,
+    OUT count int8,
+    OUT count_percentage float4,
+    OUT record_size int8,
+    OUT record_size_percentage float4,
+    OUT fpi_size int8,
+    OUT fpi_size_percentage float4,
+    OUT combined_size int8,
+    OUT combined_size_percentage float4
+)
+RETURNS SETOF record
+AS 'MODULE_PATHNAME', 'pg_get_wal_stats_till_end_of_wal'
+LANGUAGE C STRICT PARALLEL SAFE;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn, boolean) FROM PUBLIC;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats_till_end_of_wal(pg_lsn, boolean) TO pg_read_server_files;
diff --git a/contrib/pg_walinspect/pg_walinspect.c b/contrib/pg_walinspect/pg_walinspect.c
new file mode 100644
index 0000000000..6cb0a6df40
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.c
@@ -0,0 +1,650 @@
+/*-------------------------------------------------------------------------
+ *
+ * pg_walinspect.c
+ *		  Functions to inspect contents of PostgreSQL Write-Ahead Log
+ *
+ * Copyright (c) 2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *		  contrib/pg_walinspect/pg_walinspect.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/xlog.h"
+#include "access/xlog_internal.h"
+#include "access/xlogreader.h"
+#include "access/xlogrecovery.h"
+#include "access/xlogstats.h"
+#include "access/xlogutils.h"
+#include "funcapi.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+#include "utils/pg_lsn.h"
+
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_waldump tool as well.
+ */
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(pg_get_wal_record_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info);
+PG_FUNCTION_INFO_V1(pg_get_wal_records_info_till_end_of_wal);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats);
+PG_FUNCTION_INFO_V1(pg_get_wal_stats_till_end_of_wal);
+
+/*
+ * Struct holding information about the parameters that can be passed to
+ * GetWalStats.
+ */
+typedef struct GetWalStatsParams
+{
+	/* If true, generate statistics per-record instead of per-rmgr. */
+	bool stats_per_record;
+} GetWalStatsParams;
+
+typedef void (*GetWALDetailsCB) (FunctionCallInfo fcinfo,
+								 XLogRecPtr start_lsn,
+								 XLogRecPtr end_lsn,
+								 void *params);
+
+static bool IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn);
+static XLogReaderState *InitXLogReaderState(XLogRecPtr lsn,
+											XLogRecPtr *first_record);
+static XLogRecord *ReadNextXLogRecord(XLogReaderState *xlogreader,
+									  XLogRecPtr first_record);
+static void GetWALRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+							  GetWALDetailsCB wal_details_cb);
+static void GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+							  XLogRecPtr end_lsn, void *params);
+static void GetXLogSummaryStats(XLogStats * stats, ReturnSetInfo *rsinfo,
+								Datum *values, bool *nulls, uint32 ncols,
+								bool stats_per_record);
+static void FillXLogStatsRow(const char *name, uint64 n, uint64 total_count,
+							 uint64 rec_len, uint64 total_rec_len,
+							 uint64 fpi_len, uint64 total_fpi_len,
+							 uint64 tot_len, uint64 total_len,
+							 Datum *values, bool *nulls, uint32 ncols);
+static void GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+						XLogRecPtr end_lsn, void *params);
+
+/*
+ * Check if the given LSN is in future. Also, return the LSN up to which the
+ * server has WAL.
+ */
+static bool
+IsFutureLSN(XLogRecPtr lsn, XLogRecPtr *curr_lsn)
+{
+	/*
+	 * We determine the current LSN of the server similar to how page_read
+	 * callback read_local_xlog_page_no_wait does.
+	 */
+	if (!RecoveryInProgress())
+		*curr_lsn = GetFlushRecPtr(NULL);
+	else
+		*curr_lsn = GetXLogReplayRecPtr(NULL);
+
+	Assert(!XLogRecPtrIsInvalid(*curr_lsn));
+
+	if (lsn >= *curr_lsn)
+		return true;
+
+	return false;
+}
+
+/*
+ * Intialize WAL reader and identify first valid LSN.
+ */
+static XLogReaderState *
+InitXLogReaderState(XLogRecPtr lsn, XLogRecPtr *first_record)
+{
+	XLogReaderState *xlogreader;
+
+	/*
+	 * Reading WAL below the first page of the first sgements isn't allowed.
+	 * This is a bootstrap WAL page and the page_read callback fails to read
+	 * it.
+	 */
+	if (lsn < XLOG_BLCKSZ)
+		ereport(ERROR,
+				(errmsg("could not read WAL at LSN %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	xlogreader = XLogReaderAllocate(wal_segment_size, NULL,
+									XL_ROUTINE(.page_read = &read_local_xlog_page_no_wait,
+											   .segment_open = &wal_segment_open,
+											   .segment_close = &wal_segment_close),
+									NULL);
+
+	if (xlogreader == NULL)
+		ereport(ERROR,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("out of memory"),
+				 errdetail("Failed while allocating a WAL reading processor.")));
+
+	/* First find a valid recptr to start from. */
+	*first_record = XLogFindNextRecord(xlogreader, lsn);
+
+	if (XLogRecPtrIsInvalid(*first_record))
+		ereport(ERROR,
+				(errmsg("could not find a valid record after %X/%X",
+						LSN_FORMAT_ARGS(lsn))));
+
+	return xlogreader;
+}
+
+/*
+ * Read next WAL record.
+ */
+static XLogRecord *
+ReadNextXLogRecord(XLogReaderState *xlogreader, XLogRecPtr first_record)
+{
+	XLogRecord *record;
+	char	*errormsg;
+
+	record = XLogReadRecord(xlogreader, &errormsg);
+
+	if (record == NULL)
+	{
+		if (errormsg)
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X: %s",
+							LSN_FORMAT_ARGS(first_record), errormsg)));
+		else
+			ereport(ERROR,
+					(errcode_for_file_access(),
+					 errmsg("could not read WAL at %X/%X",
+							LSN_FORMAT_ARGS(first_record))));
+	}
+
+	return record;
+}
+
+/*
+ * Get a single WAL record info.
+ */
+static void
+GetWALRecordInfo(XLogReaderState *record, XLogRecPtr lsn,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	const char *id;
+	RmgrData desc;
+	uint32	fpi_len = 0;
+	StringInfoData rec_desc;
+	StringInfoData rec_blk_ref;
+	uint32	main_data_len;
+	int	i = 0;
+
+	desc = GetRmgr(XLogRecGetRmid(record));
+	id = desc.rm_identify(XLogRecGetInfo(record));
+
+	if (id == NULL)
+		id = psprintf("UNKNOWN (%x)", XLogRecGetInfo(record) & ~XLR_INFO_MASK);
+
+	initStringInfo(&rec_desc);
+	desc.rm_desc(&rec_desc, record);
+
+	/* Block references. */
+	initStringInfo(&rec_blk_ref);
+	XLogRecGetBlockRefInfo(record, NULL, &fpi_len, true, &rec_blk_ref);
+
+	main_data_len = XLogRecGetDataLen(record);
+
+	values[i++] = LSNGetDatum(lsn);
+	values[i++] = LSNGetDatum(record->EndRecPtr - 1);
+	values[i++] = LSNGetDatum(XLogRecGetPrev(record));
+	values[i++] = TransactionIdGetDatum(XLogRecGetXid(record));
+	values[i++] = CStringGetTextDatum(desc.rm_name);
+	values[i++] = CStringGetTextDatum(id);
+	values[i++] = UInt32GetDatum(XLogRecGetTotalLen(record));
+	values[i++] = UInt32GetDatum(main_data_len);
+	values[i++] = UInt32GetDatum(fpi_len);
+	values[i++] = CStringGetTextDatum(rec_desc.data);
+	values[i++] = CStringGetTextDatum(rec_blk_ref.data);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get WAL record info.
+ *
+ * This function emits an error if a future WAL LSN i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_record_info(PG_FUNCTION_ARGS)
+{
+#define PG_GET_WAL_RECORD_INFO_COLS 11
+	Datum	result;
+	Datum	values[PG_GET_WAL_RECORD_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORD_INFO_COLS];
+	XLogRecPtr	lsn;
+	XLogRecPtr	curr_lsn;
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	TupleDesc	tupdesc;
+	HeapTuple	tuple;
+
+	lsn = PG_GETARG_LSN(0);
+
+	if (IsFutureLSN(lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future input LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	/* Build a tuple descriptor for our result type. */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	xlogreader = InitXLogReaderState(lsn, &first_record);
+
+	Assert(xlogreader);
+
+	(void) ReadNextXLogRecord(xlogreader, first_record);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetWALRecordInfo(xlogreader, first_record, values, nulls,
+					 PG_GET_WAL_RECORD_INFO_COLS);
+
+	XLogReaderFree(xlogreader);
+
+	tuple = heap_form_tuple(tupdesc, values, nulls);
+	result = HeapTupleGetDatum(tuple);
+
+	PG_RETURN_DATUM(result);
+#undef PG_GET_WAL_RECORD_INFO_COLS
+}
+
+/*
+ * Get WAL details such as record info, stats using the passed in callback.
+ */
+static void
+GetWALDetailsGuts(FunctionCallInfo fcinfo, bool till_end_of_wal,
+				  GetWALDetailsCB wal_details_cb)
+{
+	XLogRecPtr	start_lsn;
+	XLogRecPtr	end_lsn;
+	XLogRecPtr	curr_lsn;
+	GetWalStatsParams stats_params;
+
+	start_lsn = PG_GETARG_LSN(0);
+
+	/* If not till end of wal, end_lsn would have been specified. */
+	if (!till_end_of_wal)
+		end_lsn = PG_GETARG_LSN(1);
+
+	if (IsFutureLSN(start_lsn, &curr_lsn))
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future start LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+
+	if (!till_end_of_wal && end_lsn >= curr_lsn)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up
+		 * to "end" in user facing message.
+		 */
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("cannot accept future end LSN"),
+				 errdetail("Last WAL record on the database system ends at LSN %X/%X.",
+						   LSN_FORMAT_ARGS(curr_lsn - 1))));
+	}
+	else if (till_end_of_wal)
+	{
+		/*
+		 * GetFlushRecPtr or GetXLogReplayRecPtr gives "end+1" LSN of the last
+		 * record flushed or replayed respectively. But let's use the LSN up to
+		 * "end".
+		 */
+		end_lsn = curr_lsn - 1;
+	}
+
+	if (start_lsn >= end_lsn)
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("WAL start LSN must be less than end LSN")));
+
+	if (wal_details_cb == GetWalStats)
+	{
+		MemSet(&stats_params, 0, sizeof(GetWalStatsParams));
+
+		if (!till_end_of_wal)
+			stats_params.stats_per_record = PG_GETARG_BOOL(2);
+		else
+			stats_params.stats_per_record = PG_GETARG_BOOL(1);
+
+		wal_details_cb(fcinfo, start_lsn, end_lsn, (void *) &stats_params);
+	}
+	else if (wal_details_cb == GetWALRecordsInfo)
+	{
+		wal_details_cb(fcinfo, start_lsn, end_lsn, NULL);
+	}
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ */
+static void
+GetWALRecordsInfo(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+				  XLogRecPtr end_lsn, void *params)
+{
+#define PG_GET_WAL_RECORDS_INFO_COLS 11
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_RECORDS_INFO_COLS];
+	bool	nulls[PG_GET_WAL_RECORDS_INFO_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	Assert(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+		{
+			GetWALRecordInfo(xlogreader, xlogreader->currRecPtr, values, nulls,
+							 PG_GET_WAL_RECORDS_INFO_COLS);
+
+			tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+								 values, nulls);
+		}
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+#undef PG_GET_WAL_RECORDS_INFO_COLS
+}
+
+/*
+ * Get info and data of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get info and data of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_records_info_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWALRecordsInfo);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Fill single row of record counts and sizes for an rmgr or record.
+ */
+static void
+FillXLogStatsRow(const char *name,
+				 uint64 n, uint64 total_count,
+				 uint64 rec_len, uint64 total_rec_len,
+				 uint64 fpi_len, uint64 total_fpi_len,
+				 uint64 tot_len, uint64 total_len,
+				 Datum *values, bool *nulls, uint32 ncols)
+{
+	double	n_pct,
+			rec_len_pct,
+			fpi_len_pct,
+			tot_len_pct;
+	int	i = 0;
+
+	n_pct = 0;
+	if (total_count != 0)
+		n_pct = 100 * (double) n / total_count;
+
+	rec_len_pct = 0;
+	if (total_rec_len != 0)
+		rec_len_pct = 100 * (double) rec_len / total_rec_len;
+
+	fpi_len_pct = 0;
+	if (total_fpi_len != 0)
+		fpi_len_pct = 100 * (double) fpi_len / total_fpi_len;
+
+	tot_len_pct = 0;
+	if (total_len != 0)
+		tot_len_pct = 100 * (double) tot_len / total_len;
+
+	values[i++] = CStringGetTextDatum(name);
+	values[i++] = Int64GetDatum(n);
+	values[i++] = Float4GetDatum(n_pct);
+	values[i++] = Int64GetDatum(rec_len);
+	values[i++] = Float4GetDatum(rec_len_pct);
+	values[i++] = Int64GetDatum(fpi_len);
+	values[i++] = Float4GetDatum(fpi_len_pct);
+	values[i++] = Int64GetDatum(tot_len);
+	values[i++] = Float4GetDatum(tot_len_pct);
+
+	Assert(i == ncols);
+}
+
+/*
+ * Get summary statistics about the records seen so far.
+ */
+static void
+GetXLogSummaryStats(XLogStats *stats, ReturnSetInfo *rsinfo,
+					Datum *values, bool *nulls, uint32 ncols,
+					bool stats_per_record)
+{
+	uint64	total_count = 0;
+	uint64	total_rec_len = 0;
+	uint64	total_fpi_len = 0;
+	uint64	total_len = 0;
+	int	ri;
+
+	/*
+	 * Each row shows its percentages of the total, so make a first pass to
+	 * calculate column totals.
+	 */
+	for (ri = 0; ri <= RM_MAX_ID; ri++)
+	{
+		if (!RmgrIdIsValid(ri))
+			continue;
+
+		total_count += stats->rmgr_stats[ri].count;
+		total_rec_len += stats->rmgr_stats[ri].rec_len;
+		total_fpi_len += stats->rmgr_stats[ri].fpi_len;
+	}
+	total_len = total_rec_len + total_fpi_len;
+
+	for (ri = 0; ri <= RM_MAX_ID; ri++)
+	{
+		uint64		count;
+		uint64		rec_len;
+		uint64		fpi_len;
+		uint64		tot_len;
+		RmgrData 	desc;
+
+		if (!RmgrIdIsValid(ri))
+			continue;
+
+		if (!RmgrIdExists(ri))
+			continue;
+
+		desc = GetRmgr(ri);
+
+		if (stats_per_record)
+		{
+			int rj;
+
+			for (rj = 0; rj < MAX_XLINFO_TYPES; rj++)
+			{
+				const char *id;
+
+				count = stats->record_stats[ri][rj].count;
+				rec_len = stats->record_stats[ri][rj].rec_len;
+				fpi_len = stats->record_stats[ri][rj].fpi_len;
+				tot_len = rec_len + fpi_len;
+
+				/* Skip undefined combinations and ones that didn't occur */
+				if (count == 0)
+					continue;
+
+				/* the upper four bits in xl_info are the rmgr's */
+				id = desc.rm_identify(rj << 4);
+				if (id == NULL)
+					id = psprintf("UNKNOWN (%x)", rj << 4);
+
+				FillXLogStatsRow(psprintf("%s/%s", desc.rm_name, id), count,
+								 total_count, rec_len, total_rec_len, fpi_len,
+								 total_fpi_len, tot_len, total_len,
+								 values, nulls, ncols);
+
+				tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+									 values, nulls);
+			}
+		}
+		else
+		{
+			count = stats->rmgr_stats[ri].count;
+			rec_len = stats->rmgr_stats[ri].rec_len;
+			fpi_len = stats->rmgr_stats[ri].fpi_len;
+			tot_len = rec_len + fpi_len;
+
+			FillXLogStatsRow(desc.rm_name, count, total_count, rec_len,
+							 total_rec_len, fpi_len, total_fpi_len, tot_len,
+							 total_len, values, nulls, ncols);
+
+			tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+								 values, nulls);
+		}
+	}
+}
+
+/*
+ * Get WAL stats between start LSN and end LSN.
+ */
+static void
+GetWalStats(FunctionCallInfo fcinfo, XLogRecPtr start_lsn,
+			XLogRecPtr end_lsn, void *params)
+{
+#define PG_GET_WAL_STATS_COLS 9
+	XLogRecPtr	first_record;
+	XLogReaderState *xlogreader;
+	XLogStats stats;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	Datum	values[PG_GET_WAL_STATS_COLS];
+	bool	nulls[PG_GET_WAL_STATS_COLS];
+
+	SetSingleFuncCall(fcinfo, 0);
+
+	xlogreader = InitXLogReaderState(start_lsn, &first_record);
+
+	MemSet(&stats, 0, sizeof(stats));
+
+	for (;;)
+	{
+		(void) ReadNextXLogRecord(xlogreader, first_record);
+
+		/*
+		 * Let's not show the record info if it is spanning more than the
+		 * end_lsn. EndRecPtr is "end+1" of the last read record, hence
+		 * use "end" here.
+		 */
+		if ((xlogreader->EndRecPtr - 1) <= end_lsn)
+			XLogRecStoreStats(&stats, xlogreader);
+
+		/* Exit loop if read up to end_lsn. */
+		if (xlogreader->EndRecPtr >= end_lsn)
+			break;
+
+		CHECK_FOR_INTERRUPTS();
+	}
+
+	XLogReaderFree(xlogreader);
+
+	MemSet(values, 0, sizeof(values));
+	MemSet(nulls, 0, sizeof(nulls));
+
+	GetXLogSummaryStats(&stats, rsinfo, values, nulls,
+						PG_GET_WAL_STATS_COLS,
+						((GetWalStatsParams *)params)->stats_per_record);
+
+#undef PG_GET_WAL_STATS_COLS
+}
+
+/*
+ * Get stats of all WAL records between start LSN and end LSN.
+ *
+ * This function emits an error if a future start or end WAL LSN i.e. WAL LSN
+ * the database system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, false, GetWalStats);
+
+	PG_RETURN_VOID();
+}
+
+/*
+ * Get stats of all WAL records from start LSN till end of WAL.
+ *
+ * This function emits an error if a future start i.e. WAL LSN the database
+ * system doesn't know about is specified.
+ */
+Datum
+pg_get_wal_stats_till_end_of_wal(PG_FUNCTION_ARGS)
+{
+	GetWALDetailsGuts(fcinfo, true, GetWalStats);
+
+	PG_RETURN_VOID();
+}
diff --git a/contrib/pg_walinspect/pg_walinspect.control b/contrib/pg_walinspect/pg_walinspect.control
new file mode 100644
index 0000000000..017e56a2bb
--- /dev/null
+++ b/contrib/pg_walinspect/pg_walinspect.control
@@ -0,0 +1,5 @@
+# pg_walinspect extension
+comment = 'functions to inspect contents of PostgreSQL Write-Ahead Log'
+default_version = '1.0'
+module_pathname = '$libdir/pg_walinspect'
+relocatable = true
diff --git a/src/backend/access/transam/xlogreader.c b/src/backend/access/transam/xlogreader.c
index 5862d9dc75..dea1f877ae 100644
--- a/src/backend/access/transam/xlogreader.c
+++ b/src/backend/access/transam/xlogreader.c
@@ -1320,13 +1320,6 @@ XLogReaderValidatePageHeader(XLogReaderState *state, XLogRecPtr recptr,
 	return true;
 }
 
-#ifdef FRONTEND
-/*
- * Functions that are currently not needed in the backend, but are better
- * implemented inside xlogreader.c because of the internal facilities available
- * here.
- */
-
 /*
  * Find the first record with an lsn >= RecPtr.
  *
@@ -1447,6 +1440,12 @@ err:
 	return InvalidXLogRecPtr;
 }
 
+#ifdef FRONTEND
+/*
+ * Functions that are currently not needed in the backend, but are better
+ * implemented inside xlogreader.c because of the internal facilities available
+ * here.
+ */
 #endif							/* FRONTEND */
 
 /*
diff --git a/src/backend/access/transam/xlogutils.c b/src/backend/access/transam/xlogutils.c
index bb2d3ec991..b5d34c61e6 100644
--- a/src/backend/access/transam/xlogutils.c
+++ b/src/backend/access/transam/xlogutils.c
@@ -80,6 +80,10 @@ typedef struct xl_invalid_page
 
 static HTAB *invalid_page_tab = NULL;
 
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal);
 
 /* Report a reference to an invalid page */
 static void
@@ -870,6 +874,31 @@ wal_segment_close(XLogReaderState *state)
 int
 read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 					 int reqLen, XLogRecPtr targetRecPtr, char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, true);
+}
+
+/*
+ * Same as read_local_xlog_page except that it doesn't wait for future WAL
+ * to be available.
+ */
+int
+read_local_xlog_page_no_wait(XLogReaderState *state, XLogRecPtr targetPagePtr,
+							 int reqLen, XLogRecPtr targetRecPtr,
+							 char *cur_page)
+{
+	return read_local_xlog_page_guts(state, targetPagePtr, reqLen,
+									 targetRecPtr, cur_page, false);
+}
+
+/*
+ * Implementation of read_local_xlog_page and its no wait version.
+ */
+static int
+read_local_xlog_page_guts(XLogReaderState *state, XLogRecPtr targetPagePtr,
+						  int reqLen, XLogRecPtr targetRecPtr,
+						  char *cur_page, bool wait_for_wal)
 {
 	XLogRecPtr	read_upto,
 				loc;
@@ -925,6 +954,10 @@ read_local_xlog_page(XLogReaderState *state, XLogRecPtr targetPagePtr,
 			if (loc <= read_upto)
 				break;
 
+			/* If asked, let's not wait for future WAL. */
+			if (!wait_for_wal)
+				break;
+
 			CHECK_FOR_INTERRUPTS();
 			pg_usleep(1000L);
 		}
diff --git a/src/bin/pg_waldump/pg_waldump.c b/src/bin/pg_waldump/pg_waldump.c
index d918845cb2..54d765b331 100644
--- a/src/bin/pg_waldump/pg_waldump.c
+++ b/src/bin/pg_waldump/pg_waldump.c
@@ -27,6 +27,11 @@
 #include "getopt_long.h"
 #include "rmgrdesc.h"
 
+/*
+ * NOTE: For any code change or issue fix here, it is highly recommended to
+ * give a thought about doing the same in pg_walinspect contrib module as well.
+ */
+
 static const char *progname;
 
 static int	WalSegSz;
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index e302bd102c..5e1e3446ae 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -31,7 +31,7 @@ extern XLogRecPtr XactLastRecEnd;
 extern PGDLLIMPORT XLogRecPtr XactLastCommitEnd;
 
 /* these variables are GUC parameters related to XLOG */
-extern int	wal_segment_size;
+extern PGDLLIMPORT int	wal_segment_size;
 extern int	min_wal_size_mb;
 extern int	max_wal_size_mb;
 extern int	wal_keep_size_mb;
diff --git a/src/include/access/xlog_internal.h b/src/include/access/xlog_internal.h
index d0206f7c74..fd8d51af7d 100644
--- a/src/include/access/xlog_internal.h
+++ b/src/include/access/xlog_internal.h
@@ -320,7 +320,7 @@ typedef struct RmgrData
 							  struct XLogRecordBuffer *buf);
 } RmgrData;
 
-extern RmgrData RmgrTable[];
+extern PGDLLIMPORT RmgrData RmgrTable[];
 extern void RmgrStartup(void);
 extern void RmgrCleanup(void);
 extern void RmgrNotFound(RmgrId rmid);
diff --git a/src/include/access/xlogreader.h b/src/include/access/xlogreader.h
index d8eb857611..727e9fe971 100644
--- a/src/include/access/xlogreader.h
+++ b/src/include/access/xlogreader.h
@@ -344,9 +344,7 @@ extern void XLogReaderSetDecodeBuffer(XLogReaderState *state,
 
 /* Position the XLogReader to given record */
 extern void XLogBeginRead(XLogReaderState *state, XLogRecPtr RecPtr);
-#ifdef FRONTEND
 extern XLogRecPtr XLogFindNextRecord(XLogReaderState *state, XLogRecPtr RecPtr);
-#endif							/* FRONTEND */
 
 /* Return values from XLogPageReadCB. */
 typedef enum XLogPageReadResult
diff --git a/src/include/access/xlogutils.h b/src/include/access/xlogutils.h
index ff40f96e42..3746e31e40 100644
--- a/src/include/access/xlogutils.h
+++ b/src/include/access/xlogutils.h
@@ -93,6 +93,10 @@ extern void FreeFakeRelcacheEntry(Relation fakerel);
 extern int	read_local_xlog_page(XLogReaderState *state,
 								 XLogRecPtr targetPagePtr, int reqLen,
 								 XLogRecPtr targetRecPtr, char *cur_page);
+extern int	read_local_xlog_page_no_wait(XLogReaderState *state,
+										 XLogRecPtr targetPagePtr, int reqLen,
+										 XLogRecPtr targetRecPtr,
+										 char *cur_page);
 extern void wal_segment_open(XLogReaderState *state,
 							 XLogSegNo nextSegNo,
 							 TimeLineID *tli_p);
-- 
2.25.1

v18-0003-pg_walinspect-tests.patchapplication/x-patch; name=v18-0003-pg_walinspect-tests.patchDownload
From 67c34743edf3a8dbbec8b7eafa8f7b9be91c8dd1 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 7 Apr 2022 06:58:02 +0000
Subject: [PATCH v18] pg_walinspect tests

---
 .../pg_walinspect/expected/pg_walinspect.out  | 145 ++++++++++++++++++
 contrib/pg_walinspect/sql/pg_walinspect.sql   | 107 +++++++++++++
 2 files changed, 252 insertions(+)
 create mode 100644 contrib/pg_walinspect/expected/pg_walinspect.out
 create mode 100644 contrib/pg_walinspect/sql/pg_walinspect.sql

diff --git a/contrib/pg_walinspect/expected/pg_walinspect.out b/contrib/pg_walinspect/expected/pg_walinspect.out
new file mode 100644
index 0000000000..53d3146fe1
--- /dev/null
+++ b/contrib/pg_walinspect/expected/pg_walinspect.out
@@ -0,0 +1,145 @@
+CREATE EXTENSION pg_walinspect;
+CREATE TABLE sample_tbl(col1 int, col2 int);
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+ERROR:  WAL start LSN must be less than end LSN
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+ ok 
+----
+ t
+(1 row)
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+ ok 
+----
+ t
+(1 row)
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- no
+ has_function_privilege 
+------------------------
+ f
+(1 row)
+
+-- Functions accessible by users with role pg_read_server_files
+GRANT pg_read_server_files TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean)
+  TO regress_pg_walinspect;
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- yes
+ has_function_privilege 
+------------------------
+ t
+(1 row)
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean)
+  FROM regress_pg_walinspect;
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+DROP ROLE regress_pg_walinspect;
+DROP TABLE sample_tbl;
diff --git a/contrib/pg_walinspect/sql/pg_walinspect.sql b/contrib/pg_walinspect/sql/pg_walinspect.sql
new file mode 100644
index 0000000000..6e120c472b
--- /dev/null
+++ b/contrib/pg_walinspect/sql/pg_walinspect.sql
@@ -0,0 +1,107 @@
+CREATE EXTENSION pg_walinspect;
+
+CREATE TABLE sample_tbl(col1 int, col2 int);
+
+SELECT pg_current_wal_lsn() AS wal_lsn1 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+SELECT pg_current_wal_lsn() AS wal_lsn2 \gset
+
+INSERT INTO sample_tbl SELECT * FROM generate_series(1, 2);
+
+-- ===================================================================
+-- Tests for input validation
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn2', :'wal_lsn1'); -- ERROR
+
+-- ===================================================================
+-- Tests for all function executions
+-- ===================================================================
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_record_info(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_records_info_till_end_of_wal(:'wal_lsn1');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats(:'wal_lsn1', :'wal_lsn2');
+
+SELECT COUNT(*) >= 0 AS ok FROM pg_get_wal_stats_till_end_of_wal(:'wal_lsn1');
+
+-- ===================================================================
+-- Tests for filtering out WAL records of a particular table
+-- ===================================================================
+
+SELECT oid AS sample_tbl_oid FROM pg_class WHERE relname = 'sample_tbl' \gset
+
+SELECT COUNT(*) >= 1 AS ok FROM pg_get_wal_records_info(:'wal_lsn1', :'wal_lsn2')
+			WHERE block_ref LIKE concat('%', :'sample_tbl_oid', '%') AND resource_manager = 'Heap';
+
+-- ===================================================================
+-- Tests for permissions
+-- ===================================================================
+CREATE ROLE regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- no
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- no
+
+-- Functions accessible by users with role pg_read_server_files
+
+GRANT pg_read_server_files TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- yes
+
+REVOKE pg_read_server_files FROM regress_pg_walinspect;
+
+-- Superuser can grant execute to other users
+GRANT EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  TO regress_pg_walinspect;
+
+GRANT EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean)
+  TO regress_pg_walinspect;
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_record_info(pg_lsn)', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_records_info(pg_lsn, pg_lsn) ', 'EXECUTE'); -- yes
+
+SELECT has_function_privilege('regress_pg_walinspect',
+  'pg_get_wal_stats(pg_lsn, pg_lsn, boolean) ', 'EXECUTE'); -- yes
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_record_info(pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_records_info(pg_lsn, pg_lsn)
+  FROM regress_pg_walinspect;
+
+REVOKE EXECUTE ON FUNCTION pg_get_wal_stats(pg_lsn, pg_lsn, boolean)
+  FROM regress_pg_walinspect;
+
+-- ===================================================================
+-- Clean up
+-- ===================================================================
+
+DROP ROLE regress_pg_walinspect;
+
+DROP TABLE sample_tbl;
-- 
2.25.1

v18-0004-pg_walinspect-docs.patchapplication/x-patch; name=v18-0004-pg_walinspect-docs.patchDownload
From e146d961755294f8064d6afd0c24a6c6d77e11b4 Mon Sep 17 00:00:00 2001
From: Bharath Rupireddy <bharath.rupireddyforpostgres@gmail.com>
Date: Thu, 7 Apr 2022 06:58:45 +0000
Subject: [PATCH v18] pg_walinspect docs

---
 doc/src/sgml/contrib.sgml      |   1 +
 doc/src/sgml/filelist.sgml     |   1 +
 doc/src/sgml/pgwalinspect.sgml | 244 +++++++++++++++++++++++++++++++++
 3 files changed, 246 insertions(+)
 create mode 100644 doc/src/sgml/pgwalinspect.sgml

diff --git a/doc/src/sgml/contrib.sgml b/doc/src/sgml/contrib.sgml
index 1e42ce1a7f..4e7b87a42f 100644
--- a/doc/src/sgml/contrib.sgml
+++ b/doc/src/sgml/contrib.sgml
@@ -131,6 +131,7 @@ CREATE EXTENSION <replaceable>module_name</replaceable>;
  &pgsurgery;
  &pgtrgm;
  &pgvisibility;
+ &pgwalinspect;
  &postgres-fdw;
  &seg;
  &sepgsql;
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index 7dea670969..1e82cb2d3d 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -148,6 +148,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
+<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
diff --git a/doc/src/sgml/pgwalinspect.sgml b/doc/src/sgml/pgwalinspect.sgml
new file mode 100644
index 0000000000..e44874b87b
--- /dev/null
+++ b/doc/src/sgml/pgwalinspect.sgml
@@ -0,0 +1,244 @@
+<!-- doc/src/sgml/pgwalinspect.sgml -->
+
+<sect1 id="pgwalinspect" xreflabel="pg_walinspect">
+ <title>pg_walinspect</title>
+
+ <indexterm zone="pgwalinspect">
+  <primary>pg_walinspect</primary>
+ </indexterm>
+
+ <para>
+  The <filename>pg_walinspect</filename> module provides functions that allow
+  you to inspect the contents of write-ahead log of <productname>PostgreSQL</productname>
+  database cluster at a low level, which is useful for debugging or analytical
+  or reporting or educational purposes.
+ </para>
+
+ <para>
+  All the functions of this module will provide the WAL information using the
+  current server's timeline ID.
+ </para>
+
+ <para>
+  By default, use of these functions is restricted to superusers and members of
+  the <literal>pg_read_server_files</literal> role. Access may be granted by
+  superusers to others using <command>GRANT</command>.
+ </para>
+    
+ <sect2>
+  <title>General Functions</title>
+
+  <variablelist>
+   <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_record_info(in_lsn pg_lsn,
+                             start_lsn OUT pg_lsn,
+                             end_lsn OUT pg_lsn,
+                             prev_lsn OUT pg_lsn,
+                             xid OUT xid,
+                             resource_manager OUT text,
+                             record_type OUT text,
+                             record_length OUT int4,
+                             main_data_length OUT int4,
+                             fpi_length OUT int4,
+                             description OUT text,
+                             block_ref OUT text)
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets WAL record information of a given LSN. If the given LSN isn't
+      containing a valid WAL record, it gives the information of the next
+      available valid WAL record. This function emits an error if a future (the
+      LSN database system doesn't know about) <replaceable>in_lsn</replaceable>
+      is specified.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info(start_lsn pg_lsn,
+                              end_lsn pg_lsn,
+                              start_lsn OUT pg_lsn,
+                              end_lsn OUT pg_lsn,
+                              prev_lsn OUT pg_lsn,
+                              xid OUT xid,
+                              resource_manager OUT text,
+                              record_type OUT text,
+                              record_length OUT int4,
+                              main_data_length OUT int4,
+                              fpi_length OUT int4,
+                              description OUT text,
+                              block_ref OUT text)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets information of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and <replaceable>end_lsn</replaceable>.
+      Returns one row per each valid WAL record. This function emits an error
+      if a future (the LSN database system doesn't know about)
+      <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select start_lsn, end_lsn, prev_lsn, xid, resource_manager, record_type, record_length, main_data_length, fpi_length, description from pg_get_wal_records_info('0/14FBA30', '0/15011D7');
+ start_lsn |  end_lsn  | prev_lsn  | xid | resource_manager |  record_type  | record_length | main_data_length | fpi_length |                       description                       
+-----------+-----------+-----------+-----+------------------+---------------+---------------+------------------+------------+---------------------------------------------------------
+ 0/14FBA30 | 0/14FBA67 | 0/14FB9F8 |   0 | Heap2            | PRUNE         |            56 |                8 |          0 | latestRemovedXid 0 nredirected 0 ndead 1
+ 0/14FBA68 | 0/14FBA9F | 0/14FBA30 |   0 | Standby          | RUNNING_XACTS |            50 |               24 |          0 | nextXid 723 latestCompletedXid 722 oldestRunningXid 723
+ 0/14FBAA0 | 0/14FBACF | 0/14FBA68 |   0 | Storage          | CREATE        |            42 |               16 |          0 | base/5/16390
+ 0/14FBAD0 | 0/14FC117 | 0/14FBAA0 | 723 | Heap             | INSERT        |          1582 |                3 |       1528 | off 8 flags 0x01
+ 0/14FC118 | 0/14FD487 | 0/14FBAD0 | 723 | Btree            | INSERT_LEAF   |          4973 |                2 |       4920 | off 244
+ 0/14FD488 | 0/14FEFCF | 0/14FC118 | 723 | Btree            | INSERT_LEAF   |          6953 |                2 |       6900 | off 126
+ 0/14FEFD0 | 0/14FF027 | 0/14FD488 | 723 | Heap2            | MULTI_INSERT  |            85 |                6 |          0 | 1 tuples flags 0x02
+ 0/14FF028 | 0/14FF06F | 0/14FEFD0 | 723 | Btree            | INSERT_LEAF   |            72 |                2 |          0 | off 155
+ 0/14FF070 | 0/14FF0B7 | 0/14FF028 | 723 | Btree            | INSERT_LEAF   |            72 |                2 |          0 | off 134
+ 0/14FF0B8 | 0/14FF18F | 0/14FF070 | 723 | Heap             | INSERT        |           211 |                3 |          0 | off 9 flags 0x00
+ 0/14FF190 | 0/14FF1CF | 0/14FF0B8 | 723 | Btree            | INSERT_LEAF   |            64 |                2 |          0 | off 244
+ 0/14FF1D0 | 0/15010B7 | 0/14FF190 | 723 | Btree            | SPLIT_L       |          7885 |               10 |       4136 | level 0, firstrightoff 120, newitemoff 47, postingoff 0
+ 0/15010B8 | 0/150117F | 0/14FF1D0 | 723 | Btree            | INSERT_UPPER  |           197 |                2 |        136 | off 2
+ 0/1501180 | 0/15011D7 | 0/15010B8 | 723 | Heap2            | MULTI_INSERT  |            85 |                6 |          0 | 1 tuples flags 0x02
+(14 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_records_info_till_end_of_wal(start_lsn pg_lsn,
+                                              start_lsn OUT pg_lsn,
+                                              end_lsn OUT pg_lsn,
+                                              prev_lsn OUT pg_lsn,
+                                              xid OUT xid,
+                                              resource_manager OUT text,
+                                              record_type OUT text,
+                                              record_length OUT int4,
+                                              main_data_length OUT int4,
+                                              fpi_length OUT int4,
+                                              description OUT text,
+                                              block_ref OUT text)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_records_info()</function>
+      except that it gets information of all the valid WAL records from 
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats(start_lsn pg_lsn,
+                       end_lsn pg_lsn,
+                       per_record boolean DEFAULT false,
+                       "resource_manager/record_type" OUT text,
+                       count OUT int8,
+                       count_percentage OUT float4,
+                       record_length OUT int8,
+                       record_length_percentage OUT float4,
+                       fpi_length OUT int8,
+                       fpi_length_percentage OUT float4,
+                       combined_size OUT int8,
+                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      Gets statistics of all the valid WAL records between
+      <replaceable>start_lsn</replaceable> and
+      <replaceable>end_lsn</replaceable>. By default, it returns one row per
+      <replaceable>resource_manager</replaceable> type. When
+      <replaceable>per_record</replaceable> is set to <literal>true</literal>,
+      it returns one row per <replaceable>record_type</replaceable>. This
+      function emits an error if a future (the LSN database system doesn't know
+      about) <replaceable>start_lsn</replaceable> or <replaceable>end_lsn</replaceable>
+      is specified. For example, usage of the function is as follows:
+<screen>
+postgres=# select * from pg_get_wal_stats('0/12FBA30', '0/15011D7') where count > 0;
+ resource_manager/record_type | count | count_percentage | record_size | record_size_percentage | fpi_size | fpi_size_percentage | combined_size | combined_size_percentage 
+------------------------------+-------+------------------+-------------+------------------------+----------+---------------------+---------------+--------------------------
+ XLOG                         |    12 |       0.13002492 |        1024 |          2.3833392e-05 |      352 |          0.06136488 |          1376 |            3.2021846e-05
+ Transaction                  |   188 |        2.0370572 |       62903 |           0.0014640546 |        0 |                   0 |         62903 |             0.0014638591
+ Storage                      |    13 |       0.14086033 |         546 |          1.2708038e-05 |        0 |                   0 |           546 |            1.2706342e-05
+ Database                     |     2 |       0.02167082 |          84 |           1.955083e-06 |        0 |                   0 |            84 |             1.954822e-06
+ Standby                      |   219 |        2.3729548 |       15830 |          0.00036844003 |        0 |                   0 |         15830 |            0.00036839084
+ Heap2                        |  1905 |        20.641457 |      384619 |           0.0089519285 |   364472 |            63.53915 |        749091 |              0.017432613
+ Heap                         |  1319 |        14.291906 |      621997 |            0.014476853 |   145232 |           25.318592 |        767229 |              0.017854715
+ Btree                        |  5571 |         60.36407 |  4295405999 |                99.9747 |    63562 |          11.0808935 |    4295469561 |                 99.96284
+(8 rows)
+</screen>
+
+With <replaceable>per_record</replaceable> passed as <literal>true</literal>:
+
+<screen>
+postgres=# select * from pg_get_wal_stats('0/14FBA30', '0/15011D7', true) where count > 0;
+ resource_manager/record_type | count | count_percentage | record_size | record_size_percentage | fpi_size | fpi_size_percentage | combined_size | combined_size_percentage 
+------------------------------+-------+------------------+-------------+------------------------+----------+---------------------+---------------+--------------------------
+ Storage/CREATE               |     1 |         7.142857 |          42 |              0.8922881 |        0 |                   0 |            42 |               0.18811305
+ Standby/RUNNING_XACTS        |     1 |         7.142857 |          50 |              1.0622478 |        0 |                   0 |            50 |                0.2239441
+ Heap2/PRUNE                  |     1 |         7.142857 |          56 |              1.1897174 |        0 |                   0 |            56 |                0.2508174
+ Heap2/MULTI_INSERT           |     2 |        14.285714 |         170 |              3.6116421 |        0 |                   0 |           170 |               0.76140994
+ Heap/INSERT                  |     2 |        14.285714 |         265 |               5.629913 |     1528 |            8.671964 |          1793 |                 8.030636
+ Btree/INSERT_LEAF            |     5 |        35.714287 |         314 |              6.6709156 |    11820 |            67.08286 |         12134 |                54.346756
+ Btree/INSERT_UPPER           |     1 |         7.142857 |          61 |              1.2959422 |      136 |          0.77185017 |           197 |                0.8823398
+ Btree/SPLIT_L                |     1 |         7.142857 |        3749 |               79.64733 |     4136 |           23.473326 |          7885 |                35.315987
+(8 rows)
+</screen>
+     </para>
+    </listitem>
+   </varlistentry>
+
+    <varlistentry>
+    <term>
+     <function>
+      pg_get_wal_stats_till_end_of_wal(start_lsn pg_lsn,
+                                       per_record boolean DEFAULT false,
+                                       "resource_manager/record_type" OUT text,
+                                       count OUT int8,
+                                       count_percentage OUT float4,
+                                       record_length OUT int8,
+                                       record_length_percentage OUT float4,
+                                       fpi_length OUT int8,
+                                       fpi_length_percentage OUT float4,
+                                       combined_size OUT int8,
+                                       combined_size_percentage OUT float4)
+      returns setof record
+     </function>
+    </term>
+
+    <listitem>
+     <para>
+      This function is same as <function>pg_get_wal_stats()</function> except
+      that it gets statistics of all the valid WAL records from
+      <replaceable>start_lsn</replaceable> till end of WAL.
+     </para>
+    </listitem>
+   </varlistentry>
+
+  </variablelist>
+ </sect2>
+
+ <sect2>
+  <title>Author</title>
+
+  <para>
+   Bharath Rupireddy <email>bharath.rupireddyforpostgres@gmail.com</email>
+  </para>
+ </sect2>
+
+</sect1>
-- 
2.25.1

#88Amit Kapila
amit.kapila16@gmail.com
In reply to: Bharath Rupireddy (#87)
1 attachment(s)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Thu, Apr 7, 2022 at 3:35 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

I am facing the below doc build failure on my machine due to this work:

./filelist.sgml:<!ENTITY pgwalinspect SYSTEM "pgwalinspect.sgml">
Tabs appear in SGML/XML files
make: *** [check-tabs] Error 1

The attached patch fixes this for me.

--
With Regards,
Amit Kapila.

Attachments:

fix_tabs_1.patchapplication/octet-stream; name=fix_tabs_1.patchDownload
diff --git a/doc/src/sgml/filelist.sgml b/doc/src/sgml/filelist.sgml
index 1e82cb2..40ef5f7 100644
--- a/doc/src/sgml/filelist.sgml
+++ b/doc/src/sgml/filelist.sgml
@@ -148,7 +148,7 @@
 <!ENTITY pgsurgery       SYSTEM "pgsurgery.sgml">
 <!ENTITY pgtrgm          SYSTEM "pgtrgm.sgml">
 <!ENTITY pgvisibility    SYSTEM "pgvisibility.sgml">
-<!ENTITY pgwalinspect 	 SYSTEM "pgwalinspect.sgml">
+<!ENTITY pgwalinspect    SYSTEM "pgwalinspect.sgml">
 <!ENTITY postgres-fdw    SYSTEM "postgres-fdw.sgml">
 <!ENTITY seg             SYSTEM "seg.sgml">
 <!ENTITY contrib-spi     SYSTEM "contrib-spi.sgml">
#89Bharath Rupireddy
bharath.rupireddyforpostgres@gmail.com
In reply to: Amit Kapila (#88)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Mon, Apr 11, 2022 at 4:21 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Thu, Apr 7, 2022 at 3:35 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

I am facing the below doc build failure on my machine due to this work:

./filelist.sgml:<!ENTITY pgwalinspect SYSTEM "pgwalinspect.sgml">
Tabs appear in SGML/XML files
make: *** [check-tabs] Error 1

The attached patch fixes this for me.

Thanks. It looks like there's a TAB in between. Your patch LGTM.

I'm wondering why this hasn't been caught in the build farm members
(or it may have been found but I'm missing to locate it.).

Can you please provide me with the doc build command to catch these
kinds of errors?

Regards,
Bharath Rupireddy.

#90Amit Kapila
amit.kapila16@gmail.com
In reply to: Bharath Rupireddy (#89)
Re: pg_walinspect - a new extension to get raw WAL data and WAL stats

On Mon, Apr 11, 2022 at 6:33 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

On Mon, Apr 11, 2022 at 4:21 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Thu, Apr 7, 2022 at 3:35 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:

I am facing the below doc build failure on my machine due to this work:

./filelist.sgml:<!ENTITY pgwalinspect SYSTEM "pgwalinspect.sgml">
Tabs appear in SGML/XML files
make: *** [check-tabs] Error 1

The attached patch fixes this for me.

Thanks. It looks like there's a TAB in between. Your patch LGTM.

I'm wondering why this hasn't been caught in the build farm members
(or it may have been found but I'm missing to locate it.).

Can you please provide me with the doc build command to catch these
kinds of errors?

Nothing special. In the doc/src/sgml, I did make clean followed by make check.

--
With Regards,
Amit Kapila.