to <literal>replica</literal> or higher to allow replication slots to
          be used.
         </para>
-
-        <para>
-         Note that this parameter also applies on the subscriber side, but with
-         a different meaning. See <xref linkend="guc-max-replication-slots-subscriber"/>
-         in <xref linkend="runtime-config-replication-subscriber"/> for more
-         details.
-        </para>
        </listitem>
       </varlistentry>
 
 
      <variablelist>
 
-     <varlistentry id="guc-max-replication-slots-subscriber" xreflabel="max_replication_slots">
-      <term><varname>max_replication_slots</varname> (<type>integer</type>)
+     <varlistentry id="guc-max-active-replication-origins" xreflabel="max_active_replication_origins">
+      <term><varname>max_active_replication_origins</varname> (<type>integer</type>)
        <indexterm>
-        <primary><varname>max_replication_slots</varname> configuration parameter</primary>
+        <primary><varname>max_active_replication_origins</varname> configuration parameter</primary>
         <secondary>in a subscriber</secondary>
        </indexterm>
       </term>
         be created on the server. Setting it to a lower value than the current
         number of tracked replication origins (reflected in
         <link linkend="view-pg-replication-origin-status">pg_replication_origin_status</link>)
-        will prevent the server from starting.
-        <literal>max_replication_slots</literal> must be set to at least the
+        will prevent the server from starting. It defaults to 10. This parameter
+        can only be set at server start.
+
+        <literal>max_active_replication_origins</literal> must be set to at least the
         number of subscriptions that will be added to the subscriber, plus some
         reserve for table synchronization.
        </para>
-
-       <para>
-        Note that this parameter also applies on a sending server, but with
-        a different meaning. See <xref linkend="guc-max-replication-slots"/>
-        in <xref linkend="runtime-config-replication-sender"/> for more
-        details.
-       </para>
       </listitem>
      </varlistentry>
 
 
   <title>Configuration Settings</title>
 
   <para>
-   Logical replication requires several configuration options to be set. Most
-   options are relevant only on one side of the replication. However,
-   <varname>max_replication_slots</varname> is used on both the publisher and
-   the subscriber, but it has a different meaning for each.
+   Logical replication requires several configuration options to be set. These
+   options are relevant only on one side of the replication.
   </para>
 
   <sect2 id="logical-replication-config-publisher">
    <title>Subscribers</title>
 
    <para>
-    <link linkend="guc-max-replication-slots-subscriber"><varname>max_replication_slots</varname></link>
+    <link linkend="guc-max-active-replication-origins"><varname>max_active_replication_origins</varname></link>
     must be set to at least the number of subscriptions that will be added to
     the subscriber, plus some reserve for table synchronization.
    </para>
     <listitem>
      <para>
       The new cluster must have
-      <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+      <link linkend="guc-max-active-replication-origins"><varname>max_active_replication_origins</varname></link>
       configured to a value greater than or equal to the number of
       subscriptions present in the old cluster.
      </para>
 
 
    <para>
     The target server must be used as a physical standby.  The target server
-    must have <xref linkend="guc-max-replication-slots"/> and <xref
+    must have <xref linkend="guc-max-active-replication-origins"/> and <xref
     linkend="guc-max-logical-replication-workers"/> configured to a value
     greater than or equal to the number of specified databases.  The target
     server must have <xref linkend="guc-max-worker-processes"/> configured to a
 
 #include "postmaster/bgworker.h"
 #include "postmaster/interrupt.h"
 #include "replication/logicallauncher.h"
-#include "replication/slot.h"
+#include "replication/origin.h"
 #include "replication/walreceiver.h"
 #include "replication/worker_internal.h"
 #include "storage/ipc.h"
                             subname)));
 
    /* Report this after the initial starting message for consistency. */
-   if (max_replication_slots == 0)
+   if (max_active_replication_origins == 0)
        ereport(ERROR,
                (errcode(ERRCODE_CONFIGURATION_LIMIT_EXCEEDED),
-                errmsg("cannot start logical replication workers when \"max_replication_slots\"=0")));
+                errmsg("cannot start logical replication workers when \"max_active_replication_origins\"=0")));
 
    /*
     * We need to do the modification of the shared memory under lock so that
 
 #include "storage/lmgr.h"
 #include "utils/builtins.h"
 #include "utils/fmgroids.h"
+#include "utils/guc.h"
 #include "utils/pg_lsn.h"
 #include "utils/rel.h"
 #include "utils/snapmgr.h"
 #define PG_REPLORIGIN_CHECKPOINT_FILENAME PG_LOGICAL_DIR "/replorigin_checkpoint"
 #define PG_REPLORIGIN_CHECKPOINT_TMPFILE PG_REPLORIGIN_CHECKPOINT_FILENAME ".tmp"
 
+/* GUC variables */
+int            max_active_replication_origins = 10;
+
 /*
  * Replay progress of a single remote node.
  */
 {
    /* Tranche to use for per-origin LWLocks */
    int         tranche_id;
-   /* Array of length max_replication_slots */
+   /* Array of length max_active_replication_origins */
    ReplicationState states[FLEXIBLE_ARRAY_MEMBER];
 } ReplicationStateCtl;
 
 
 /*
  * Base address into a shared memory array of replication states of size
- * max_replication_slots.
- *
- * XXX: Should we use a separate variable to size this rather than
- * max_replication_slots?
+ * max_active_replication_origins.
  */
 static ReplicationState *replication_states;
 
 #define REPLICATION_STATE_MAGIC ((uint32) 0x1257DADE)
 
 static void
-replorigin_check_prerequisites(bool check_slots, bool recoveryOK)
+replorigin_check_prerequisites(bool check_origins, bool recoveryOK)
 {
-   if (check_slots && max_replication_slots == 0)
+   if (check_origins && max_active_replication_origins == 0)
        ereport(ERROR,
                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-                errmsg("cannot query or manipulate replication origin when \"max_replication_slots\" is 0")));
+                errmsg("cannot query or manipulate replication origin when \"max_active_replication_origins\" is 0")));
 
    if (!recoveryOK && RecoveryInProgress())
        ereport(ERROR,
 restart:
    LWLockAcquire(ReplicationOriginLock, LW_EXCLUSIVE);
 
-   for (i = 0; i < max_replication_slots; i++)
+   for (i = 0; i < max_active_replication_origins; i++)
    {
        ReplicationState *state = &replication_states[i];
 
 {
    Size        size = 0;
 
-   /*
-    * XXX: max_replication_slots is arguably the wrong thing to use, as here
-    * we keep the replay state of *remote* transactions. But for now it seems
-    * sufficient to reuse it, rather than introduce a separate GUC.
-    */
-   if (max_replication_slots == 0)
+   if (max_active_replication_origins == 0)
        return size;
 
    size = add_size(size, offsetof(ReplicationStateCtl, states));
 
    size = add_size(size,
-                   mul_size(max_replication_slots, sizeof(ReplicationState)));
+                   mul_size(max_active_replication_origins, sizeof(ReplicationState)));
    return size;
 }
 
 {
    bool        found;
 
-   if (max_replication_slots == 0)
+   if (max_active_replication_origins == 0)
        return;
 
    replication_states_ctl = (ReplicationStateCtl *)
 
        replication_states_ctl->tranche_id = LWTRANCHE_REPLICATION_ORIGIN_STATE;
 
-       for (i = 0; i < max_replication_slots; i++)
+       for (i = 0; i < max_active_replication_origins; i++)
        {
            LWLockInitialize(&replication_states[i].lock,
                             replication_states_ctl->tranche_id);
  *
  * So its just the magic, followed by the statically sized
  * ReplicationStateOnDisk structs. Note that the maximum number of
- * ReplicationState is determined by max_replication_slots.
+ * ReplicationState is determined by max_active_replication_origins.
  * ---------------------------------------------------------------------------
  */
 void
    uint32      magic = REPLICATION_STATE_MAGIC;
    pg_crc32c   crc;
 
-   if (max_replication_slots == 0)
+   if (max_active_replication_origins == 0)
        return;
 
    INIT_CRC32C(crc);
    LWLockAcquire(ReplicationOriginLock, LW_SHARED);
 
    /* write actual data */
-   for (i = 0; i < max_replication_slots; i++)
+   for (i = 0; i < max_active_replication_origins; i++)
    {
        ReplicationStateOnDisk disk_state;
        ReplicationState *curstate = &replication_states[i];
    already_started = true;
 #endif
 
-   if (max_replication_slots == 0)
+   if (max_active_replication_origins == 0)
        return;
 
    INIT_CRC32C(crc);
    fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
 
    /*
-    * might have had max_replication_slots == 0 last run, or we just brought
-    * up a standby.
+    * might have had max_active_replication_origins == 0 last run, or we just
+    * brought up a standby.
     */
    if (fd < 0 && errno == ENOENT)
        return;
 
        COMP_CRC32C(crc, &disk_state, sizeof(disk_state));
 
-       if (last_state == max_replication_slots)
+       if (last_state == max_active_replication_origins)
            ereport(PANIC,
                    (errcode(ERRCODE_CONFIGURATION_LIMIT_EXCEEDED),
-                    errmsg("could not find free replication state, increase \"max_replication_slots\"")));
+                    errmsg("could not find free replication state, increase \"max_active_replication_origins\"")));
 
        /* copy data to shared memory */
        replication_states[last_state].roident = disk_state.roident;
 
                xlrec = (xl_replorigin_drop *) XLogRecGetData(record);
 
-               for (i = 0; i < max_replication_slots; i++)
+               for (i = 0; i < max_active_replication_origins; i++)
                {
                    ReplicationState *state = &replication_states[i];
 
     * Search for either an existing slot for the origin, or a free one we can
     * use.
     */
-   for (i = 0; i < max_replication_slots; i++)
+   for (i = 0; i < max_active_replication_origins; i++)
    {
        ReplicationState *curstate = &replication_states[i];
 
                (errcode(ERRCODE_CONFIGURATION_LIMIT_EXCEEDED),
                 errmsg("could not find free replication state slot for replication origin with ID %d",
                        node),
-                errhint("Increase \"max_replication_slots\" and try again.")));
+                errhint("Increase \"max_active_replication_origins\" and try again.")));
 
    if (replication_state == NULL)
    {
    /* prevent slots from being concurrently dropped */
    LWLockAcquire(ReplicationOriginLock, LW_SHARED);
 
-   for (i = 0; i < max_replication_slots; i++)
+   for (i = 0; i < max_active_replication_origins; i++)
    {
        ReplicationState *state;
 
        registered_cleanup = true;
    }
 
-   Assert(max_replication_slots > 0);
+   Assert(max_active_replication_origins > 0);
 
    if (session_replication_state != NULL)
        ereport(ERROR,
     * Search for either an existing slot for the origin, or a free one we can
     * use.
     */
-   for (i = 0; i < max_replication_slots; i++)
+   for (i = 0; i < max_active_replication_origins; i++)
    {
        ReplicationState *curstate = &replication_states[i];
 
                (errcode(ERRCODE_CONFIGURATION_LIMIT_EXCEEDED),
                 errmsg("could not find free replication state slot for replication origin with ID %d",
                        node),
-                errhint("Increase \"max_replication_slots\" and try again.")));
+                errhint("Increase \"max_active_replication_origins\" and try again.")));
    else if (session_replication_state == NULL)
    {
        /* initialize new slot */
 {
    ConditionVariable *cv;
 
-   Assert(max_replication_slots != 0);
+   Assert(max_active_replication_origins != 0);
 
    if (session_replication_state == NULL)
        ereport(ERROR,
     * filled. Note that we do not take any locks, so slightly corrupted/out
     * of date values are a possibility.
     */
-   for (i = 0; i < max_replication_slots; i++)
+   for (i = 0; i < max_active_replication_origins; i++)
    {
        ReplicationState *state;
        Datum       values[REPLICATION_ORIGIN_PROGRESS_COLS];
 
        NULL, NULL, NULL
    },
 
+   {
+       {"max_active_replication_origins",
+           PGC_POSTMASTER,
+           REPLICATION_SUBSCRIBERS,
+           gettext_noop("Sets the maximum number of active replication origins."),
+           NULL
+       },
+       &max_active_replication_origins,
+       10, 0, MAX_BACKENDS,
+       NULL, NULL, NULL
+   },
+
    {
        {"log_rotation_age", PGC_SIGHUP, LOGGING_WHERE,
            gettext_noop("Sets the amount of time to wait before forcing "
 
 
 # These settings are ignored on a publisher.
 
+#max_active_replication_origins = 10   # max number of active replication origins
+                   # (change requires restart)
 #max_logical_replication_workers = 4   # taken from max_worker_processes
                    # (change requires restart)
 #max_sync_workers_per_subscription = 2 # taken from max_logical_replication_workers
 
    bool        failed = false;
 
    int         max_lrworkers;
-   int         max_repslots;
+   int         max_reporigins;
    int         max_wprocs;
 
    pg_log_info("checking settings on subscriber");
     * Since these parameters are not a requirement for physical replication,
     * we should check it to make sure it won't fail.
     *
-    * - max_replication_slots >= number of dbs to be converted
+    * - max_active_replication_origins >= number of dbs to be converted
     * - max_logical_replication_workers >= number of dbs to be converted
     * - max_worker_processes >= 1 + number of dbs to be converted
     *------------------------------------------------------------------------
    res = PQexec(conn,
                 "SELECT setting FROM pg_catalog.pg_settings WHERE name IN ("
                 "'max_logical_replication_workers', "
-                "'max_replication_slots', "
+                "'max_active_replication_origins', "
                 "'max_worker_processes', "
                 "'primary_slot_name') "
                 "ORDER BY name");
        disconnect_database(conn, true);
    }
 
-   max_lrworkers = atoi(PQgetvalue(res, 0, 0));
-   max_repslots = atoi(PQgetvalue(res, 1, 0));
+   max_reporigins = atoi(PQgetvalue(res, 0, 0));
+   max_lrworkers = atoi(PQgetvalue(res, 1, 0));
    max_wprocs = atoi(PQgetvalue(res, 2, 0));
    if (strcmp(PQgetvalue(res, 3, 0), "") != 0)
        primary_slot_name = pg_strdup(PQgetvalue(res, 3, 0));
 
    pg_log_debug("subscriber: max_logical_replication_workers: %d",
                 max_lrworkers);
-   pg_log_debug("subscriber: max_replication_slots: %d", max_repslots);
+   pg_log_debug("subscriber: max_active_replication_origins: %d", max_reporigins);
    pg_log_debug("subscriber: max_worker_processes: %d", max_wprocs);
    if (primary_slot_name)
        pg_log_debug("subscriber: primary_slot_name: %s", primary_slot_name);
 
    disconnect_database(conn, false);
 
-   if (max_repslots < num_dbs)
+   if (max_reporigins < num_dbs)
    {
-       pg_log_error("subscriber requires %d replication slots, but only %d remain",
-                    num_dbs, max_repslots);
+       pg_log_error("subscriber requires %d active replication origins, but only %d remain",
+                    num_dbs, max_reporigins);
        pg_log_error_hint("Increase the configuration parameter \"%s\" to at least %d.",
-                         "max_replication_slots", num_dbs);
+                         "max_active_replication_origins", num_dbs);
        failed = true;
    }
 
 
 # Check some unmet conditions on node S
 $node_s->append_conf(
    'postgresql.conf', q{
-max_replication_slots = 1
+max_active_replication_origins = 1
 max_logical_replication_workers = 1
 max_worker_processes = 2
 });
    'standby contains unmet conditions on node S');
 $node_s->append_conf(
    'postgresql.conf', q{
-max_replication_slots = 10
+max_active_replication_origins = 10
 max_logical_replication_workers = 4
 max_worker_processes = 8
 });
 
 /*
  * check_new_cluster_subscription_configuration()
  *
- * Verify that the max_replication_slots configuration specified is enough for
- * creating the subscriptions. This is required to create the replication
- * origin for each subscription.
+ * Verify that the max_active_replication_origins configuration specified is
+ * enough for creating the subscriptions. This is required to create the
+ * replication origin for each subscription.
  */
 static void
 check_new_cluster_subscription_configuration(void)
 {
    PGresult   *res;
    PGconn     *conn;
-   int         max_replication_slots;
+   int         max_active_replication_origins;
 
    /* Subscriptions and their dependencies can be migrated since PG17. */
    if (GET_MAJOR_VERSION(old_cluster.major_version) < 1700)
    conn = connectToServer(&new_cluster, "template1");
 
    res = executeQueryOrDie(conn, "SELECT setting FROM pg_settings "
-                           "WHERE name = 'max_replication_slots';");
+                           "WHERE name = 'max_active_replication_origins';");
 
    if (PQntuples(res) != 1)
        pg_fatal("could not determine parameter settings on new cluster");
 
-   max_replication_slots = atoi(PQgetvalue(res, 0, 0));
-   if (old_cluster.nsubs > max_replication_slots)
-       pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+   max_active_replication_origins = atoi(PQgetvalue(res, 0, 0));
+   if (old_cluster.nsubs > max_active_replication_origins)
+       pg_fatal("\"max_active_replication_origins\" (%d) must be greater than or equal to the number of "
                 "subscriptions (%d) on the old cluster",
-                max_replication_slots, old_cluster.nsubs);
+                max_active_replication_origins, old_cluster.nsubs);
 
    PQclear(res);
    PQfinish(conn);
 
 my $connstr = $publisher->connstr . ' dbname=postgres';
 
 # ------------------------------------------------------
-# Check that pg_upgrade fails when max_replication_slots configured in the new
-# cluster is less than the number of subscriptions in the old cluster.
+# Check that pg_upgrade fails when max_active_replication_origins configured
+# in the new cluster is less than the number of subscriptions in the old
+# cluster.
 # ------------------------------------------------------
 # It is sufficient to use disabled subscription to test upgrade failure.
 $publisher->safe_psql('postgres', "CREATE PUBLICATION regress_pub1");
 
 $old_sub->stop;
 
-$new_sub->append_conf('postgresql.conf', "max_replication_slots = 0");
+$new_sub->append_conf('postgresql.conf', "max_active_replication_origins = 0");
 
 # pg_upgrade will fail because the new cluster has insufficient
-# max_replication_slots.
+# max_active_replication_origins.
 command_checks_all(
    [
        'pg_upgrade',
    ],
    1,
    [
-       qr/"max_replication_slots" \(0\) must be greater than or equal to the number of subscriptions \(1\) on the old cluster/
+       qr/"max_active_replication_origins" \(0\) must be greater than or equal to the number of subscriptions \(1\) on the old cluster/
    ],
    [qr//],
-   'run of pg_upgrade where the new cluster has insufficient max_replication_slots'
+   'run of pg_upgrade where the new cluster has insufficient max_active_replication_origins'
 );
 
-# Reset max_replication_slots
-$new_sub->append_conf('postgresql.conf', "max_replication_slots = 10");
+# Reset max_active_replication_origins
+$new_sub->append_conf('postgresql.conf', "max_active_replication_origins = 10");
 
 # Cleanup
 $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
 
 extern PGDLLIMPORT XLogRecPtr replorigin_session_origin_lsn;
 extern PGDLLIMPORT TimestampTz replorigin_session_origin_timestamp;
 
+/* GUCs */
+extern PGDLLIMPORT int max_active_replication_origins;
+
 /* API for querying & manipulating replication origins */
 extern RepOriginId replorigin_by_name(const char *roname, bool missing_ok);
 extern RepOriginId replorigin_create(const char *roname);