Back
December 19, 2023
8
min read

Using Stand-by Servers for Postgres Logical Replication

By
Gunnar Morling
Share this post

For users of Change Data Capture (CDC), one of the most exciting features in Postgres version 16 (released in September this year) is the support for logical replication from stand-by servers. Instead of connecting to your primary server, you can now point CDC tools such as Debezium to a replica server, which is very interesting for instance from a load distribution perspective. I am going to take a closer look at this new feature in this two-part blog series:

  • Part I (this part): Explaining what Postgres logical replication is, why to use it on stand-by servers, and how to set it up
  • Part II: Discussing how to use logical replication on Postgres stand-bys with Debezium and how to handle failover scenarios

What is Postgres Logical Replication?

Let’s start with the fundamentals. Replication is the process of synchronizing all the data from one database server to one or more other servers, for the purposes of ensuring high availability (HA)—if the primary server fails, one of the stand-bys can take over—and load distribution, as replicas can serve read requests. When it comes to replication in Postgres, it supports two kinds of continuous replication from primary to replicas: streaming and logical replication.

  • Streaming replication (sometimes also called physical replication) propagates all the segments of the Write-Ahead Log (WAL) from a primary server to one or more stand-by servers (or replicas). It includes all the databases and tables and automatically propagates schema changes, resulting in an exact copy of the primary server. This makes it a great choice for ensuring high availability of your data, in particular when used in synchronous mode where writing transactions to the primary server are only committed after successful replication.
  • In contrast, logical replication works much more selectively, “replicating data objects and their changes, based upon their replication identity (usually a primary key)”. It operates in the context of one single database on a Postgres server, and you can control on a very fine-grained level which schemas or tables should be part of the replication stream.

Logical replication offers a flexible publish/subscribe model, which goes beyond the basic primary/read-replica scheme of physical replication. In this model, a publisher node is a Postgres instance which exposes one or more publications, describing which tables should be replicated. Publications also allow you to limit which columns, rows, and operations (<span class="inline-code">INSERT</span>, <span class="inline-code">UPDATE</span>, or <span class="inline-code">DELETE</span>) should be propagated. Once set up, one or more subscriber nodes retrieve the changes from a publication.

A subscriber can consume the changes from multiple publishers, and it can also be a publisher itself. You can execute local write transactions on subscribers, which means that logical replication can form the foundation for multi-master architectures. Unlike streaming replication, you can use logical replication between Postgres instances with different versions, making it a useful tool for zero-downtime version upgrades.

On the downside, there are some limitations: most importantly, logical replication does not propagate DDL changes—i.e. you must make sure by yourself that the database schemas of source and destination are in sync. Also replicating sequences, (materialized) views, and large objects is currently not supported.

What makes logical replication so interesting, beyond the purposes of just replicating data from one Postgres instance to another, is the notion of logical decoding plug-ins. They control the data format used for the logical replication messages and allow external (i.e. non-Postgres) consumers to subscribe to a replication stream. This is how tools such as Debezium implement change data capture (CDC) for Postgres, enabling use cases such as the replication of data to data warehouses and data lakes, cache invalidation, low latency synchronization of search indexes, and many others.

Having established what logical replication is and why it is such an important part of the data engineering toolbox, let’s explore why it is so interesting that Postgres now supports logical replication from stand-by servers and how to set this up.

Why Use Logical Replication On Stand-By Servers?

The first and most important reason for using logical replication from stand-by servers is load distribution. When you have many logical replication slots and corresponding clients, this potentially can create an unduly heavy load on the primary database server. In that case it can be beneficial to set up the replication slots on one or even multiple read replicas of the primary, thus distributing resource consumption (CPU, network bandwidth) across multiple machines.

In addition, setting up replication slots on stand-by servers is one step towards enabling failover slots, i.e. the ability for logical replication consumers to resume processing after a failure of the primary and subsequent promotion of a stand-by server. While Postgres does not (yet) support failover slots, you can implement them yourself with a bit of glue code, as we’ll discuss in the second part of this blog series.

Finally, in some cases people just don’t like the idea of logical replication clients, in particular non-Postgres tools like Debezium, directly connecting to their operational database. Whether rightfully so or not, setting up logical replication on read replicas just helps with peace of mind.

On the flip-side, there are some implications of this to consider. One is the slightly increased end-to-end latency: as changes are traveling from the primary server to the stand-by and then to the client, things will take a bit longer than when connecting your replication consumers directly to the primary. The other aspect is that read replicas are exactly that—read-only. While CDC in general doesn’t require write access to the database, there are some exceptions to that. In case of Debezium, you won’t be able to use its incremental snapshotting implementation, as this requires inserting watermarking events into a signaling table (you can learn more about this in my talk “Debezium Snapshots Revisited!”). Also you cannot use the heartbeat feature, which lets the connector update its restart offsets also in cases where there are no change events for any table the connector is capturing are coming in. If you want to use these features, then you’ll need to create a replication slot on the primary server.

Enough with all the theory, let’s have a look at logical replication on stand-bys in practice, starting with setting it up.

Provisioning a Testing Environment

As a testing environment I am going to use Postgres 16 on Amazon RDS (engine version “PostgreSQL 16.1-R1” at the time of writing), as this makes setting up a Postgres primary and an associated read replica a matter of just a few clicks. A free tier instance type like t4g.micro will do for this experiment. Instead of the pre-configured Multi-AZ cluster deployment option (which isn’t available in the free tier anyways), a single DB instance can be used, with a manually created replica.

Note that you must have backups enabled for the primary server, as otherwise the option for creating a read replica will be disabled in the RDS console. In order to use logical replication, the database’s WAL level must be set to logical. To do so, set <span class="inline-code">rds.logical_replication</span> to 1 in the DB parameter groups of primary and stand-by, as I’ve recently discussed in Data Streaming Quick Tips episode #4. Finally, you should set <span class="inline-code">hot_standby_feedback</span> to true on the stand-by server. Otherwise, the primary server might prematurely discard tuples for system catalog rows which are still needed on the stand-by for logical decoding.

<div class="side-note">Enabling hot stand-by feedback may cause a large amount of WAL segments to be pinned on the primary when the stand-by replication slot is not consumed. In the worst case this can cause the primary server to run out of disk space, clearly something you’d want to avoid at all costs. While this situation generally can be avoided by putting proper monitoring and alerting into place and has further been improved with the addition of the max_wal_keep_size setting in Postgres 13 (at the price that a replication consumer may not able to resume its work after a prolonged downtime), it is something to keep in mind when setting up logical replication on read replicas.</div>

If you would like to work with a local set-up instead, you can follow the steps from this blog post for setting up two Postgres nodes via Docker, or use any of the Postgres Kubernetes operators for a Kube-based set-up. Set <span class="inline-code">wal_level</span> to logical in the postgresql.conf of the stand-by in this case.

Testing Things Out

With our database cluster up and running, let’s take a look at the existing replication slots on the primary and on the stand-by server:

primary> WITH node_status AS (
 SELECT
   CASE WHEN pg_is_in_recovery() = 'True' Then 'stand-by' ELSE 'primary' END AS role
)
SELECT
  node_status.role as node,
  slot_name,
  slot_type,
  active,
  plugin,
  database,
  confirmed_flush_lsn
FROM
  pg_replication_slots, node_status;
  
+---------+------------------------------------------------+-----------+--------+--------+----------+---------------------+
| node    | slot_name                                      | slot_type | active | plugin | database | confirmed_flush_lsn |
|---------+------------------------------------------------+-----------+--------+--------+----------+---------------------|
| primary | rds_eu_central_1_db_eygryp2llpzqewobw57j5mrtum | physical  | True   | <null> | <null>   | <null>              |
+---------+------------------------------------------------+-----------+--------+--------+----------+---------------------+

As expected, there is one physical (i.e. streaming) replication slot on the primary. This slot has been set up automatically when creating a replica on RDS and it propagates all the changes from the primary to the stand-by server. On the stand-by server itself, there’s no replication slot at this point. So let’s create a logical replication slot on the stand-by using <span class="inline-code">pg_create_logical_replication_slot()</span>:

standby> SELECT * FROM pg_create_logical_replication_slot('demo_slot_standby', 'test_decoding');

+-------------------+-------------+
| slot_name         | lsn         |
|-------------------+-------------|
| demo_slot_standby | 5B/D80000E8 |
+-------------------+-------------+

The test_decoding plug-in which is used here, comes with Postgres out of the box and emits changes in a simple text-based representation. It allows you to examine the change events using Postgres’ logical decoding SQL interface, which comes in handy for testing logical replication without setting up a replication consumer.

As Bertrand Drouvot discusses in this blog post, the creation of the slot may take a while if there’s no traffic on the primary database: the slot will only be created when the next <span class="inline-code">xl_running_xact</span> record (describing currently active transactions) is received. Postgres 16 therefore adds a new function <span class="inline-code">pg_log_standby_snapshot()</span>, which can be invoked on the primary to trigger such a record. Unfortunately, it failed with a permission error when I tried to invoke it on RDS:

primary> SELECT pg_log_standby_snapshot();

permission denied for function pg_log_standby_snapshot

So if you are on RDS, and there are not a lot of transactions on your primary, you may have to wait for a minute or two for the slot to be created. Once that’s done, running the query from above again should show the new slot:

+----------+-------------------+-----------+--------+---------------+-----------+---------------------+
| node     | slot_name         | slot_type | active | plugin        | database  | confirmed_flush_lsn |
|----------+-------------------+-----------+--------+---------------+-----------+---------------------|
| stand-by | demo_slot_standby | logical   | False  | test_decoding | inventory | 5E/700010B8         |
+----------+-------------------+-----------+--------+---------------+-----------+---------------------+

Now it’s time to do some data changes on the primary then. Let’s create a simple table on the primary:

primary> CREATE TABLE some_data (
  id SERIAL NOT NULL PRIMARY KEY,
  short_text VARCHAR(255) NOT NULL,
  long_text text
);
ALTER TABLE some_data REPLICA IDENTITY FULL;

The table’s replica identity is set to FULL, ensuring that all its columns are part of the replication messages. 

Insert a few rows with random data using the <span class="inline-code">generate_series()</span> function:


primary> INSERT INTO some_data(short_text, long_text) 
           SELECT left(md5(i::text), 5),
                  md5(random()::text)
           FROM generate_series(1, 5) s(i);

To confirm that streaming replication from the primary to the stand-by server works, query that data on the stand-by:

standby> SELECT * FROM some_data;
+----+------------+----------------------------------+
| id | short_text | long_text                        |
|----+------------+----------------------------------|
| 1  | c4ca4      | 22f3cc8011bb8e2f553f8af1c5db18be |
| 2  | c81e7      | 205304c828220a5aea30d5a13af4a01f |
| 3  | eccbc      | c049b5cdd131fbdceb6b3172dfe7399e |
| 4  | a87ff      | 0aed3b4e9d9e1a50e65597bdec7dfbc6 |
| 5  | e4da3      | 68db5e5b4a57e7189f48cc89936567c2 |
+----+------------+----------------------------------+

Still on the replica, we can now retrieve the corresponding change events from the logical replication slot with help of the <span class="inline-code">pg_logical_slot_get_changes()</span> function. The function takes the following parameters:

  • <span class="inline-code">slot_name</span>: The name of the replication slot to get changes from,
  • <span class="inline-code">upto_lsn</span>: The latest LSN (log sequence number, i.e. an offset in the WAL) to fetch,
  • <span class="inline-code">upto_nchanges</span>: The maximum number of events to retrieve,
  • <span class="inline-code">options</span>: A variadic text parameter for specifying any decoding plug-in specific options

If neither of <span class="inline-code">upto_lsn</span> or <span class="inline-code">upto_nchanges</span> are specified, the WAL is consumed until the end. So let’s retrieve all changes for the slot:

standby> SELECT * FROM pg_logical_slot_get_changes('demo_slot_standby', NULL, NULL);
+-------------+------+----------------------------------------------------------------------------------------------------------------------------------------+
| lsn         | xid  | data                                                                                                                                   |
|-------------+------+----------------------------------------------------------------------------------------------------------------------------------------|
| 5E/90005120 | 6983 | BEGIN 6983                                                                                                                             |
| 5E/90005188 | 6983 | table public.some_data: INSERT: id[integer]:1 short_text[character varying]:'c4ca4' long_text[text]:'3a3c3274941c83e253ebf8d2438ea5a2' |
| 5E/90005290 | 6983 | table public.some_data: INSERT: id[integer]:2 short_text[character varying]:'c81e7' long_text[text]:'9d6d86f986523accd08b372333e77b4f' |
| 5E/90005338 | 6983 | table public.some_data: INSERT: id[integer]:3 short_text[character varying]:'eccbc' long_text[text]:'8848fd2b0bd6fbddbbb72091658c047d' |
| 5E/900053E0 | 6983 | table public.some_data: INSERT: id[integer]:4 short_text[character varying]:'a87ff' long_text[text]:'d7ce7d90eacaee6b8f8390023fa0636f' |
| 5E/90005488 | 6983 | table public.some_data: INSERT: id[integer]:5 short_text[character varying]:'e4da3' long_text[text]:'843ea832ec1f7adc2f23647379435982' |
| 5E/90005560 | 6983 | COMMIT 6983                                                                                                                            |
+-------------+------+----------------------------------------------------------------------------------------------------------------------------------------+

Very nice, change events retrieved from a replication slot on a stand-by server—things work as expected!

This concludes part one of this blog post series where we have explored what Postgres logical replication is, what you can do with it, and why and how to use it on stand-by servers, as possible since Postgres version 16. If you’d like to know how to use logical replication on stand-bys with Debezium and how to manage replication slots in failover scenarios, then check out the second part of this series!

Many thanks to Bertrand Drouvot, Robert Metzger, Robin Moffatt, Gwen Shapira, and Sharon Xie for their feedback while writing this post.

📫 Email signup 👇

Did you enjoy this issue of Checkpoint Chronicle? Would you like the next edition delivered directly to your email to read from the comfort of your own home?

Simply enter your email address here and we'll send you the next issue as soon as it's published—and nothing else, we promise!

👍 Got it!
Oops! Something went wrong while submitting the form.
Gunnar Morling

Gunnar is an open-source enthusiast at heart, currently working on Apache Flink-based stream processing. In his prior role as a software engineer at Red Hat, he led the Debezium project, a distributed platform for change data capture. He is a Java Champion and has founded multiple open source projects such as JfrUnit, kcctl, and MapStruct.

For users of Change Data Capture (CDC), one of the most exciting features in Postgres version 16 (released in September this year) is the support for logical replication from stand-by servers. Instead of connecting to your primary server, you can now point CDC tools such as Debezium to a replica server, which is very interesting for instance from a load distribution perspective. I am going to take a closer look at this new feature in this two-part blog series:

  • Part I (this part): Explaining what Postgres logical replication is, why to use it on stand-by servers, and how to set it up
  • Part II: Discussing how to use logical replication on Postgres stand-bys with Debezium and how to handle failover scenarios

What is Postgres Logical Replication?

Let’s start with the fundamentals. Replication is the process of synchronizing all the data from one database server to one or more other servers, for the purposes of ensuring high availability (HA)—if the primary server fails, one of the stand-bys can take over—and load distribution, as replicas can serve read requests. When it comes to replication in Postgres, it supports two kinds of continuous replication from primary to replicas: streaming and logical replication.

  • Streaming replication (sometimes also called physical replication) propagates all the segments of the Write-Ahead Log (WAL) from a primary server to one or more stand-by servers (or replicas). It includes all the databases and tables and automatically propagates schema changes, resulting in an exact copy of the primary server. This makes it a great choice for ensuring high availability of your data, in particular when used in synchronous mode where writing transactions to the primary server are only committed after successful replication.
  • In contrast, logical replication works much more selectively, “replicating data objects and their changes, based upon their replication identity (usually a primary key)”. It operates in the context of one single database on a Postgres server, and you can control on a very fine-grained level which schemas or tables should be part of the replication stream.

Logical replication offers a flexible publish/subscribe model, which goes beyond the basic primary/read-replica scheme of physical replication. In this model, a publisher node is a Postgres instance which exposes one or more publications, describing which tables should be replicated. Publications also allow you to limit which columns, rows, and operations (<span class="inline-code">INSERT</span>, <span class="inline-code">UPDATE</span>, or <span class="inline-code">DELETE</span>) should be propagated. Once set up, one or more subscriber nodes retrieve the changes from a publication.

A subscriber can consume the changes from multiple publishers, and it can also be a publisher itself. You can execute local write transactions on subscribers, which means that logical replication can form the foundation for multi-master architectures. Unlike streaming replication, you can use logical replication between Postgres instances with different versions, making it a useful tool for zero-downtime version upgrades.

On the downside, there are some limitations: most importantly, logical replication does not propagate DDL changes—i.e. you must make sure by yourself that the database schemas of source and destination are in sync. Also replicating sequences, (materialized) views, and large objects is currently not supported.

What makes logical replication so interesting, beyond the purposes of just replicating data from one Postgres instance to another, is the notion of logical decoding plug-ins. They control the data format used for the logical replication messages and allow external (i.e. non-Postgres) consumers to subscribe to a replication stream. This is how tools such as Debezium implement change data capture (CDC) for Postgres, enabling use cases such as the replication of data to data warehouses and data lakes, cache invalidation, low latency synchronization of search indexes, and many others.

Having established what logical replication is and why it is such an important part of the data engineering toolbox, let’s explore why it is so interesting that Postgres now supports logical replication from stand-by servers and how to set this up.

Why Use Logical Replication On Stand-By Servers?

The first and most important reason for using logical replication from stand-by servers is load distribution. When you have many logical replication slots and corresponding clients, this potentially can create an unduly heavy load on the primary database server. In that case it can be beneficial to set up the replication slots on one or even multiple read replicas of the primary, thus distributing resource consumption (CPU, network bandwidth) across multiple machines.

In addition, setting up replication slots on stand-by servers is one step towards enabling failover slots, i.e. the ability for logical replication consumers to resume processing after a failure of the primary and subsequent promotion of a stand-by server. While Postgres does not (yet) support failover slots, you can implement them yourself with a bit of glue code, as we’ll discuss in the second part of this blog series.

Finally, in some cases people just don’t like the idea of logical replication clients, in particular non-Postgres tools like Debezium, directly connecting to their operational database. Whether rightfully so or not, setting up logical replication on read replicas just helps with peace of mind.

On the flip-side, there are some implications of this to consider. One is the slightly increased end-to-end latency: as changes are traveling from the primary server to the stand-by and then to the client, things will take a bit longer than when connecting your replication consumers directly to the primary. The other aspect is that read replicas are exactly that—read-only. While CDC in general doesn’t require write access to the database, there are some exceptions to that. In case of Debezium, you won’t be able to use its incremental snapshotting implementation, as this requires inserting watermarking events into a signaling table (you can learn more about this in my talk “Debezium Snapshots Revisited!”). Also you cannot use the heartbeat feature, which lets the connector update its restart offsets also in cases where there are no change events for any table the connector is capturing are coming in. If you want to use these features, then you’ll need to create a replication slot on the primary server.

Enough with all the theory, let’s have a look at logical replication on stand-bys in practice, starting with setting it up.

Provisioning a Testing Environment

As a testing environment I am going to use Postgres 16 on Amazon RDS (engine version “PostgreSQL 16.1-R1” at the time of writing), as this makes setting up a Postgres primary and an associated read replica a matter of just a few clicks. A free tier instance type like t4g.micro will do for this experiment. Instead of the pre-configured Multi-AZ cluster deployment option (which isn’t available in the free tier anyways), a single DB instance can be used, with a manually created replica.

Note that you must have backups enabled for the primary server, as otherwise the option for creating a read replica will be disabled in the RDS console. In order to use logical replication, the database’s WAL level must be set to logical. To do so, set <span class="inline-code">rds.logical_replication</span> to 1 in the DB parameter groups of primary and stand-by, as I’ve recently discussed in Data Streaming Quick Tips episode #4. Finally, you should set <span class="inline-code">hot_standby_feedback</span> to true on the stand-by server. Otherwise, the primary server might prematurely discard tuples for system catalog rows which are still needed on the stand-by for logical decoding.

<div class="side-note">Enabling hot stand-by feedback may cause a large amount of WAL segments to be pinned on the primary when the stand-by replication slot is not consumed. In the worst case this can cause the primary server to run out of disk space, clearly something you’d want to avoid at all costs. While this situation generally can be avoided by putting proper monitoring and alerting into place and has further been improved with the addition of the max_wal_keep_size setting in Postgres 13 (at the price that a replication consumer may not able to resume its work after a prolonged downtime), it is something to keep in mind when setting up logical replication on read replicas.</div>

If you would like to work with a local set-up instead, you can follow the steps from this blog post for setting up two Postgres nodes via Docker, or use any of the Postgres Kubernetes operators for a Kube-based set-up. Set <span class="inline-code">wal_level</span> to logical in the postgresql.conf of the stand-by in this case.

Testing Things Out

With our database cluster up and running, let’s take a look at the existing replication slots on the primary and on the stand-by server:

primary> WITH node_status AS (
 SELECT
   CASE WHEN pg_is_in_recovery() = 'True' Then 'stand-by' ELSE 'primary' END AS role
)
SELECT
  node_status.role as node,
  slot_name,
  slot_type,
  active,
  plugin,
  database,
  confirmed_flush_lsn
FROM
  pg_replication_slots, node_status;
  
+---------+------------------------------------------------+-----------+--------+--------+----------+---------------------+
| node    | slot_name                                      | slot_type | active | plugin | database | confirmed_flush_lsn |
|---------+------------------------------------------------+-----------+--------+--------+----------+---------------------|
| primary | rds_eu_central_1_db_eygryp2llpzqewobw57j5mrtum | physical  | True   | <null> | <null>   | <null>              |
+---------+------------------------------------------------+-----------+--------+--------+----------+---------------------+

As expected, there is one physical (i.e. streaming) replication slot on the primary. This slot has been set up automatically when creating a replica on RDS and it propagates all the changes from the primary to the stand-by server. On the stand-by server itself, there’s no replication slot at this point. So let’s create a logical replication slot on the stand-by using <span class="inline-code">pg_create_logical_replication_slot()</span>:

standby> SELECT * FROM pg_create_logical_replication_slot('demo_slot_standby', 'test_decoding');

+-------------------+-------------+
| slot_name         | lsn         |
|-------------------+-------------|
| demo_slot_standby | 5B/D80000E8 |
+-------------------+-------------+

The test_decoding plug-in which is used here, comes with Postgres out of the box and emits changes in a simple text-based representation. It allows you to examine the change events using Postgres’ logical decoding SQL interface, which comes in handy for testing logical replication without setting up a replication consumer.

As Bertrand Drouvot discusses in this blog post, the creation of the slot may take a while if there’s no traffic on the primary database: the slot will only be created when the next <span class="inline-code">xl_running_xact</span> record (describing currently active transactions) is received. Postgres 16 therefore adds a new function <span class="inline-code">pg_log_standby_snapshot()</span>, which can be invoked on the primary to trigger such a record. Unfortunately, it failed with a permission error when I tried to invoke it on RDS:

primary> SELECT pg_log_standby_snapshot();

permission denied for function pg_log_standby_snapshot

So if you are on RDS, and there are not a lot of transactions on your primary, you may have to wait for a minute or two for the slot to be created. Once that’s done, running the query from above again should show the new slot:

+----------+-------------------+-----------+--------+---------------+-----------+---------------------+
| node     | slot_name         | slot_type | active | plugin        | database  | confirmed_flush_lsn |
|----------+-------------------+-----------+--------+---------------+-----------+---------------------|
| stand-by | demo_slot_standby | logical   | False  | test_decoding | inventory | 5E/700010B8         |
+----------+-------------------+-----------+--------+---------------+-----------+---------------------+

Now it’s time to do some data changes on the primary then. Let’s create a simple table on the primary:

primary> CREATE TABLE some_data (
  id SERIAL NOT NULL PRIMARY KEY,
  short_text VARCHAR(255) NOT NULL,
  long_text text
);
ALTER TABLE some_data REPLICA IDENTITY FULL;

The table’s replica identity is set to FULL, ensuring that all its columns are part of the replication messages. 

Insert a few rows with random data using the <span class="inline-code">generate_series()</span> function:


primary> INSERT INTO some_data(short_text, long_text) 
           SELECT left(md5(i::text), 5),
                  md5(random()::text)
           FROM generate_series(1, 5) s(i);

To confirm that streaming replication from the primary to the stand-by server works, query that data on the stand-by:

standby> SELECT * FROM some_data;
+----+------------+----------------------------------+
| id | short_text | long_text                        |
|----+------------+----------------------------------|
| 1  | c4ca4      | 22f3cc8011bb8e2f553f8af1c5db18be |
| 2  | c81e7      | 205304c828220a5aea30d5a13af4a01f |
| 3  | eccbc      | c049b5cdd131fbdceb6b3172dfe7399e |
| 4  | a87ff      | 0aed3b4e9d9e1a50e65597bdec7dfbc6 |
| 5  | e4da3      | 68db5e5b4a57e7189f48cc89936567c2 |
+----+------------+----------------------------------+

Still on the replica, we can now retrieve the corresponding change events from the logical replication slot with help of the <span class="inline-code">pg_logical_slot_get_changes()</span> function. The function takes the following parameters:

  • <span class="inline-code">slot_name</span>: The name of the replication slot to get changes from,
  • <span class="inline-code">upto_lsn</span>: The latest LSN (log sequence number, i.e. an offset in the WAL) to fetch,
  • <span class="inline-code">upto_nchanges</span>: The maximum number of events to retrieve,
  • <span class="inline-code">options</span>: A variadic text parameter for specifying any decoding plug-in specific options

If neither of <span class="inline-code">upto_lsn</span> or <span class="inline-code">upto_nchanges</span> are specified, the WAL is consumed until the end. So let’s retrieve all changes for the slot:

standby> SELECT * FROM pg_logical_slot_get_changes('demo_slot_standby', NULL, NULL);
+-------------+------+----------------------------------------------------------------------------------------------------------------------------------------+
| lsn         | xid  | data                                                                                                                                   |
|-------------+------+----------------------------------------------------------------------------------------------------------------------------------------|
| 5E/90005120 | 6983 | BEGIN 6983                                                                                                                             |
| 5E/90005188 | 6983 | table public.some_data: INSERT: id[integer]:1 short_text[character varying]:'c4ca4' long_text[text]:'3a3c3274941c83e253ebf8d2438ea5a2' |
| 5E/90005290 | 6983 | table public.some_data: INSERT: id[integer]:2 short_text[character varying]:'c81e7' long_text[text]:'9d6d86f986523accd08b372333e77b4f' |
| 5E/90005338 | 6983 | table public.some_data: INSERT: id[integer]:3 short_text[character varying]:'eccbc' long_text[text]:'8848fd2b0bd6fbddbbb72091658c047d' |
| 5E/900053E0 | 6983 | table public.some_data: INSERT: id[integer]:4 short_text[character varying]:'a87ff' long_text[text]:'d7ce7d90eacaee6b8f8390023fa0636f' |
| 5E/90005488 | 6983 | table public.some_data: INSERT: id[integer]:5 short_text[character varying]:'e4da3' long_text[text]:'843ea832ec1f7adc2f23647379435982' |
| 5E/90005560 | 6983 | COMMIT 6983                                                                                                                            |
+-------------+------+----------------------------------------------------------------------------------------------------------------------------------------+

Very nice, change events retrieved from a replication slot on a stand-by server—things work as expected!

This concludes part one of this blog post series where we have explored what Postgres logical replication is, what you can do with it, and why and how to use it on stand-by servers, as possible since Postgres version 16. If you’d like to know how to use logical replication on stand-bys with Debezium and how to manage replication slots in failover scenarios, then check out the second part of this series!

Many thanks to Bertrand Drouvot, Robert Metzger, Robin Moffatt, Gwen Shapira, and Sharon Xie for their feedback while writing this post.

📫 Email signup 👇

Did you enjoy this issue of Checkpoint Chronicle? Would you like the next edition delivered directly to your email to read from the comfort of your own home?

Simply enter your email address here and we'll send you the next issue as soon as it's published—and nothing else, we promise!

Gunnar Morling

Gunnar is an open-source enthusiast at heart, currently working on Apache Flink-based stream processing. In his prior role as a software engineer at Red Hat, he led the Debezium project, a distributed platform for change data capture. He is a Java Champion and has founded multiple open source projects such as JfrUnit, kcctl, and MapStruct.