From 84113fa036cff9e3c835a2a574f20a2ebfbfe9da Mon Sep 17 00:00:00 2001
From: Masahiko Sawada <sawada.mshk@gmail.com>
Date: Thu, 23 Jan 2025 16:36:37 -0800
Subject: [PATCH v2] PoC: Convert wal_level a PGC_SIGHUP parameter.

---
 src/backend/access/rmgrdesc/xlogdesc.c        |  10 +
 src/backend/access/transam/Makefile           |   1 +
 src/backend/access/transam/meson.build        |   1 +
 src/backend/access/transam/xlog.c             |  89 ++-
 src/backend/access/transam/xlogfuncs.c        |   2 +-
 src/backend/access/transam/xloglevelworker.c  | 642 ++++++++++++++++++
 src/backend/access/transam/xlogrecovery.c     |   5 +
 src/backend/commands/publicationcmds.c        |   2 +-
 src/backend/postmaster/bgworker.c             |   4 +
 src/backend/postmaster/checkpointer.c         |   7 +
 src/backend/postmaster/pgarch.c               |  26 +
 src/backend/postmaster/postmaster.c           |  20 +-
 src/backend/replication/logical/decode.c      |  12 +
 src/backend/replication/logical/logical.c     |   2 +-
 src/backend/replication/logical/slotsync.c    |  10 +-
 src/backend/replication/slot.c                |   6 +-
 src/backend/replication/walsender.c           |  29 +
 src/backend/storage/ipc/ipci.c                |   2 +
 src/backend/storage/ipc/procsignal.c          |   4 +
 src/backend/storage/ipc/standby.c             |   6 +-
 .../utils/activity/wait_event_names.txt       |   3 +
 src/backend/utils/init/postinit.c             |   3 +
 src/backend/utils/misc/guc_tables.c           |   4 +-
 src/bin/pg_controldata/pg_controldata.c       |   4 +
 src/include/access/xlog.h                     |  54 +-
 src/include/access/xloglevelworker.h          |  18 +
 src/include/catalog/pg_control.h              |   1 +
 src/include/postmaster/pgarch.h               |   1 +
 src/include/replication/walsender.h           |   1 +
 src/include/storage/lwlocklist.h              |   1 +
 src/include/storage/procsignal.h              |   2 +
 src/include/utils/guc_hooks.h                 |   2 +
 32 files changed, 927 insertions(+), 47 deletions(-)
 create mode 100644 src/backend/access/transam/xloglevelworker.c
 create mode 100644 src/include/access/xloglevelworker.h

diff --git a/src/backend/access/rmgrdesc/xlogdesc.c b/src/backend/access/rmgrdesc/xlogdesc.c
index 58040f28656..742a4c8b34e 100644
--- a/src/backend/access/rmgrdesc/xlogdesc.c
+++ b/src/backend/access/rmgrdesc/xlogdesc.c
@@ -164,6 +164,13 @@ xlog_desc(StringInfo buf, XLogReaderState *record)
 	{
 		int			wal_level;
 
+		memcpy(&wal_level, rec, sizeof(int));
+		appendStringInfo(buf, "wal_level %s", get_wal_level_string(wal_level));
+	}
+	else if (info == XLOG_WAL_LEVEL_CHANGE)
+	{
+		int			wal_level;
+
 		memcpy(&wal_level, rec, sizeof(int));
 		appendStringInfo(buf, "wal_level %s", get_wal_level_string(wal_level));
 	}
@@ -218,6 +225,9 @@ xlog_identify(uint8 info)
 		case XLOG_CHECKPOINT_REDO:
 			id = "CHECKPOINT_REDO";
 			break;
+		case XLOG_WAL_LEVEL_CHANGE:
+			id = "WAL_LEVEL_CHANGE";
+			break;
 	}
 
 	return id;
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 661c55a9db7..e98719aba73 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -32,6 +32,7 @@ OBJS = \
 	xlogbackup.o \
 	xlogfuncs.o \
 	xloginsert.o \
+	xloglevelworker.o \
 	xlogprefetcher.o \
 	xlogreader.o \
 	xlogrecovery.o \
diff --git a/src/backend/access/transam/meson.build b/src/backend/access/transam/meson.build
index e8ae9b13c8e..27947c8b902 100644
--- a/src/backend/access/transam/meson.build
+++ b/src/backend/access/transam/meson.build
@@ -20,6 +20,7 @@ backend_sources += files(
   'xlogbackup.c',
   'xlogfuncs.c',
   'xloginsert.c',
+  'xloglevelworker.c',
   'xlogprefetcher.c',
   'xlogrecovery.c',
   'xlogstats.c',
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 9c270e7d466..9a71b77ee1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -6147,6 +6147,18 @@ StartupXLOG(void)
 	Insert->fullPageWrites = lastFullPageWrites;
 	UpdateFullPageWrites();
 
+	/*
+	 * Update wal_level in shared memory.
+	 *
+	 * We use the wal_level value configured in this server rather than the
+	 * value in the ControlFile. We don't write a XLOG_WAL_LEVEL_CHANGE record
+	 * as the recovery still in-progress, but that change would be included in
+	 * the XLOG_PARAMETER_CHANGE record written by the subsequent
+	 * XLogReportParameters().
+	 */
+	if (performedWalRecovery)
+		UpdateWalLevelAfterRecovery(ControlFile->wal_level);
+
 	/*
 	 * Emit checkpoint or end-of-recovery record in XLOG, if required.
 	 */
@@ -7022,7 +7034,7 @@ CreateCheckPoint(int flags)
 	WALInsertLockAcquireExclusive();
 
 	checkPoint.fullPageWrites = Insert->fullPageWrites;
-	checkPoint.wal_level = wal_level;
+	checkPoint.wal_level = GetActiveWalLevel();
 
 	if (shutdown)
 	{
@@ -7076,9 +7088,11 @@ CreateCheckPoint(int flags)
 	 */
 	if (!shutdown)
 	{
+		int			level = GetActiveWalLevel();
+
 		/* Include WAL level in record for WAL summarizer's benefit. */
 		XLogBeginInsert();
-		XLogRegisterData((char *) &wal_level, sizeof(wal_level));
+		XLogRegisterData((char *) &level, sizeof(level));
 		(void) XLogInsert(RM_XLOG_ID, XLOG_CHECKPOINT_REDO);
 
 		/*
@@ -7403,7 +7417,7 @@ CreateEndOfRecoveryRecord(void)
 		elog(ERROR, "can only be used to end recovery");
 
 	xlrec.end_time = GetCurrentTimestamp();
-	xlrec.wal_level = wal_level;
+	xlrec.wal_level = GetActiveWalLevel();
 
 	WALInsertLockAcquireExclusive();
 	xlrec.ThisTimeLineID = XLogCtl->InsertTimeLineID;
@@ -8532,7 +8546,7 @@ xlog_redo(XLogReaderState *record)
 		 */
 		if (InRecovery && InHotStandby &&
 			xlrec.wal_level < WAL_LEVEL_LOGICAL &&
-			wal_level >= WAL_LEVEL_LOGICAL)
+			GetActiveWalLevel() >= WAL_LEVEL_LOGICAL)
 			InvalidateObsoleteReplicationSlots(RS_INVAL_WAL_LEVEL,
 											   0, InvalidOid,
 											   InvalidTransactionId);
@@ -8604,6 +8618,25 @@ xlog_redo(XLogReaderState *record)
 	{
 		/* nothing to do here, just for informational purposes */
 	}
+	else if (info == XLOG_WAL_LEVEL_CHANGE)
+	{
+		WalLevel	new_wal_level;
+
+		memcpy(&new_wal_level, XLogRecGetData(record), sizeof(new_wal_level));
+
+		if (InRecovery && InHotStandby &&
+			new_wal_level < WAL_LEVEL_LOGICAL &&
+			GetActiveWalLevel() >= WAL_LEVEL_LOGICAL)
+			InvalidateObsoleteReplicationSlots(RS_INVAL_WAL_LEVEL,
+											   0, InvalidOid,
+											   InvalidTransactionId);
+
+		/* Update our copy of the wal_level parameter in pg_control */
+		LWLockAcquire(ControlFileLock, LW_EXCLUSIVE);
+		ControlFile->wal_level = new_wal_level;
+		UpdateControlFile();
+		LWLockRelease(ControlFileLock);
+	}
 }
 
 /*
@@ -8825,11 +8858,11 @@ do_pg_backup_start(const char *backupidstr, bool fast, List **tablespaces,
 	 * During recovery, we don't need to check WAL level. Because, if WAL
 	 * level is not sufficient, it's impossible to get here during recovery.
 	 */
-	if (!backup_started_in_recovery && !XLogIsNeeded())
+	if (!backup_started_in_recovery && GetActiveWalLevel() < WAL_LEVEL_REPLICA)
 		ereport(ERROR,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 				 errmsg("WAL level not sufficient for making an online backup"),
-				 errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start.")));
+				 errhint("\"wal_level\" must be set to \"replica\" or \"logical\".")));
 
 	if (strlen(backupidstr) > MAXPGPATH)
 		ereport(ERROR,
@@ -9155,17 +9188,13 @@ do_pg_backup_stop(BackupState *state, bool waitforarchive)
 
 	Assert(state != NULL);
 
-	backup_stopped_in_recovery = RecoveryInProgress();
-
 	/*
-	 * During recovery, we don't need to check WAL level. Because, if WAL
-	 * level is not sufficient, it's impossible to get here during recovery.
+	 * This backend must have already called do_pg_backup_start(). The WAL
+	 * level check should have been done there.
 	 */
-	if (!backup_stopped_in_recovery && !XLogIsNeeded())
-		ereport(ERROR,
-				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
-				 errmsg("WAL level not sufficient for making an online backup"),
-				 errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start.")));
+	Assert(get_backup_status() == SESSION_BACKUP_RUNNING);
+
+	backup_stopped_in_recovery = RecoveryInProgress();
 
 	/*
 	 * OK to update backup counter and session-level lock.
@@ -9452,6 +9481,36 @@ register_persistent_abort_backup_handler(void)
 	already_done = true;
 }
 
+/*
+ * Wait for all running backups to finish.
+ *
+ * Note that this function doesn't have any interlock to prevent new
+ * backups from being executed.
+ */
+void
+wait_for_backup_finish(void)
+{
+	for (;;)
+	{
+		int			running_backups;
+
+		CHECK_FOR_INTERRUPTS();
+
+		WALInsertLockAcquireExclusive();
+		running_backups = XLogCtl->Insert.runningBackups;
+		WALInsertLockRelease();
+
+		if (running_backups == 0)
+			break;
+
+		WaitLatch(MyLatch,
+				  WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
+				  1000L,
+				  WAIT_EVENT_BACKUP_TERMINATION);
+		ResetLatch(MyLatch);
+	}
+}
+
 /*
  * Get latest WAL insert pointer
  */
diff --git a/src/backend/access/transam/xlogfuncs.c b/src/backend/access/transam/xlogfuncs.c
index 8c3090165f0..82e4c43f185 100644
--- a/src/backend/access/transam/xlogfuncs.c
+++ b/src/backend/access/transam/xlogfuncs.c
@@ -242,7 +242,7 @@ pg_create_restore_point(PG_FUNCTION_ARGS)
 				 errmsg("recovery is in progress"),
 				 errhint("WAL control functions cannot be executed during recovery.")));
 
-	if (!XLogIsNeeded())
+	if (GetActiveWalLevel() < WAL_LEVEL_REPLICA)
 		ereport(ERROR,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 				 errmsg("WAL level not sufficient for creating a restore point"),
diff --git a/src/backend/access/transam/xloglevelworker.c b/src/backend/access/transam/xloglevelworker.c
new file mode 100644
index 00000000000..12883413a33
--- /dev/null
+++ b/src/backend/access/transam/xloglevelworker.c
@@ -0,0 +1,642 @@
+/*-------------------------------------------------------------------------
+ * xloglevelworker.c
+ *		Functionality for controlling 'wal_level' value online.
+ *
+ * This module implements dynamic WAL level changes via SIGHUP signal,
+ * eliminating the need for server restarts. The main idea is to decouple
+ * two aspects that were previously controlled by a single wal_level
+ * setting: the information included in WAL records and the functionalities
+ * available at each WAL level.
+ *
+ * To increase the WAL level, we first allow processes to write WAL records containing
+ * the additional information required by the target functionality, while keeping
+ * these functionalities unavailable. Once all processes have synchronized to
+ * generate the WAL records, the WAL level is increased further to enable
+ * the new functionalities.
+ *
+ * Decreasing the WAL level follows a similar pattern but requires additional steps.
+ * First, any functionality that won't be supported at the lower level must be
+ * terminated. For instance, decreasing from 'replica' to 'logical' requires
+ * invalidating all logical replication slots. After termination, processes reduce
+ * their WAL information content, and once all processes are synchronized, the WAL
+ * level is decreased.
+ *
+ *
+ * Portions Copyright (c) 1996-2025, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ *	  src/backend/access/transam/xloglevelworker.c
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/parallel.h"
+#include "access/xact.h"
+#include "access/xloginsert.h"
+#include "access/xloglevelworker.h"
+#include "catalog/pg_control.h"
+#include "miscadmin.h"
+#include "postmaster/bgworker.h"
+#include "postmaster/bgwriter.h"
+#include "postmaster/interrupt.h"
+#include "postmaster/pgarch.h"
+#include "replication/slot.h"
+#include "storage/ipc.h"
+#include "storage/lmgr.h"
+#include "storage/procsignal.h"
+#include "tcop/tcopprot.h"
+#include "utils/guc_hooks.h"
+
+typedef struct WalLevelCtlData
+{
+	/*
+	 * current_wal_level is the value used by all backends to know the active
+	 * wal_level, which affects what information is logged in WAL records and
+	 * which WAL-related features such as replication, WAL archiving and
+	 * logical decoding can be used in the system.
+	 *
+	 * The process-local 'wal_level' value, which is updated when changing
+	 * wal_level GUC parameter, might not represent the up-to-date WAL level.
+	 * We need to use GetActiveWalLevel() to get the active WAL level.
+	 */
+	WalLevel	current_wal_level;
+
+	/*
+	 * Different WAL level if we're in the process of changing WAL level
+	 * online. Otherwise, same as current_wal_level.
+	 */
+	WalLevel	target_wal_level;
+
+	/* A valid pid if a worker is working */
+	pid_t		worker_pid;
+}			WalLevelCtlData;
+static WalLevelCtlData * WalLevelCtl = NULL;
+
+typedef void (*wal_level_decrease_action_cb) (void);
+typedef void (*wal_level_increase_action_cb) (void);
+
+bool		WalLevelInitialized = false;
+
+/*
+ * Both doStandbyInfoLogging and doLogicalInfoLogging are backend-local
+ * caches.
+ *
+ * They can be used to determine if information required by hot standby
+ * and/or logical decoding need to be written in WAL records. This local
+ * cache is updated when (1) process startup and (2) processing the
+ * global barrier.
+ */
+static bool doStandbyInfoLogging = false;
+static bool doLogicalInfoLogging = false;
+
+static void update_wal_logging_info_local(void);
+static void update_wal_level(WalLevel new_wal_level, bool notify_all);
+static void write_wal_level_change(WalLevel new_wal_level);
+static void increase_to_replica_action(void);
+static void increase_wal_level(WalLevel next_level, WalLevel target_level,
+							   wal_level_increase_action_cb action_cb);
+static void decrease_to_replica_action(void);
+static void decrease_to_minimal_action(void);
+static void decrease_wal_level(WalLevel next_level, WalLevel target_level,
+							   wal_level_decrease_action_cb action_cb);
+static void wal_level_control_worker_shutdown(int code, Datum arg);
+static const char *get_wal_level_string(WalLevel level);
+
+Size
+WalLevelCtlShmemSize(void)
+{
+	return sizeof(WalLevelCtlData);
+}
+
+void
+WalLevelCtlShmemInit(void)
+{
+	bool		found;
+
+	WalLevelCtl = ShmemInitStruct("wal_level control",
+								  WalLevelCtlShmemSize(),
+								  &found);
+
+	if (!found)
+	{
+		WalLevelCtl->current_wal_level = WAL_LEVEL_REPLICA;
+		WalLevelCtl->target_wal_level = WAL_LEVEL_REPLICA;
+		WalLevelCtl->worker_pid = InvalidPid;
+	}
+}
+
+/*
+ * Initialize the global wal_level. This function is called after processing
+ * the configuration at startup.
+ */
+void
+InitializeWalLevelCtl(void)
+{
+	Assert(!WalLevelInitialized);
+
+	WalLevelCtl->current_wal_level = wal_level;
+	WalLevelCtl->target_wal_level = wal_level;
+	WalLevelInitialized = true;
+}
+
+/*
+ * Update both doStandbyInfoLogging and doLogicalInfoLogging based on the
+ * current global WAL level value.
+ */
+static void
+update_wal_logging_info_local(void)
+{
+	LWLockAcquire(WalLevelControlLock, LW_SHARED);
+	doStandbyInfoLogging =
+		(WalLevelCtl->current_wal_level >= WAL_LEVEL_STANDBY_INFO_LOGGING);
+	doLogicalInfoLogging =
+		(WalLevelCtl->current_wal_level >= WAL_LEVEL_LOGICAL_INFO_LOGGING);
+	LWLockRelease(WalLevelControlLock);
+}
+
+/*
+ * Initialize process-local xlog info status. This must be called during the
+ * process startup time.
+ */
+void
+InitWalLoggingState(void)
+{
+	update_wal_logging_info_local();
+}
+
+/*
+ * This function is called when we are ordered to update the local state
+ * by a ProcSignalBarrier.
+ */
+bool
+ProcessBarrierUpdateWalLoggingState(void)
+{
+	update_wal_logging_info_local();
+	return true;
+}
+
+/*
+ * Return true if the logical info logging is enabled.
+ */
+bool
+LogicalInfoLoggingEnabled(void)
+{
+	return doLogicalInfoLogging;
+}
+
+/*
+ * Return true if the standby info logging is enabled.
+ */
+bool
+StandbyInfoLoggingEnabled(void)
+{
+	return doStandbyInfoLogging;
+}
+
+/*
+ * Return the active wal_level stored on the shared memory, WalLevelCtl.
+ */
+int
+GetActiveWalLevel(void)
+{
+	WalLevel	level;
+
+	LWLockAcquire(WalLevelControlLock, LW_SHARED);
+	level = WalLevelCtl->current_wal_level;
+	LWLockRelease(WalLevelControlLock);
+
+	return level;
+}
+
+/*
+ * Return true if the logical decoding is ready to use.
+ */
+bool
+LogicalDecodingEnabled(void)
+{
+	return GetActiveWalLevel() >= WAL_LEVEL_LOGICAL;
+}
+
+/*
+ * Update WAL level after the recovery if necessary.
+ *
+ * This function must be called ONCE at the end of the recovery.
+ */
+void
+UpdateWalLevelAfterRecovery(WalLevel level)
+{
+	if (wal_level == level)
+		return;
+
+	WalLevelCtl->current_wal_level = level;
+	UpdateWalLevel();
+
+	/* Order all processes to reflect the new WAL level */
+	WaitForProcSignalBarrier(EmitProcSignalBarrier(PROCSIGNAL_BARRIER_UPDATE_WAL_LOGGING_STATE));
+}
+
+void
+UpdateWalLevel(void)
+{
+	BackgroundWorker bgw;
+	BackgroundWorkerHandle *bgw_handle;
+
+	if (WalLevelCtl->current_wal_level == wal_level)
+		return;
+
+	/*
+	 * During recovery, we don't need to any coordination for WAL level
+	 * changing with running processes as any writes are not permitted yet.
+	 */
+	if (RecoveryInProgress())
+	{
+		WalLevelCtl->current_wal_level = wal_level;
+		return;
+	}
+
+	LWLockAcquire(WalLevelControlLock, LW_EXCLUSIVE);
+
+	/* Return if a wal-level control worker is already running */
+	if (WalLevelCtl->worker_pid != InvalidPid)
+	{
+		LWLockRelease(WalLevelControlLock);
+		return;
+	}
+
+	WalLevelCtl->target_wal_level = wal_level;
+	LWLockRelease(WalLevelControlLock);
+
+	/* Register the new dynamic worker */
+	memset(&bgw, 0, sizeof(bgw));
+	bgw.bgw_flags = BGWORKER_SHMEM_ACCESS;
+	bgw.bgw_start_time = BgWorkerStart_RecoveryFinished;
+	snprintf(bgw.bgw_library_name, MAXPGPATH, "postgres");
+	snprintf(bgw.bgw_function_name, BGW_MAXLEN, "WalLevelCtlWorkerMain");
+	snprintf(bgw.bgw_name, BGW_MAXLEN, "wal level control worker");
+	snprintf(bgw.bgw_type, BGW_MAXLEN, "wal level control worker");
+	bgw.bgw_restart_time = 5;
+	bgw.bgw_notify_pid = MyProcPid;
+
+	/*
+	 * XXX: Perhaps it's not okay that we failed to launch a bgworker and give
+	 * up wal_level change because we already reported that the change has
+	 * been accepted. Do we need to use aux process instead for that purpose?
+	 */
+	if (!RegisterDynamicBackgroundWorker(&bgw, &bgw_handle))
+		ereport(WARNING,
+				(errcode(ERRCODE_CONFIGURATION_LIMIT_EXCEEDED),
+				 errmsg("out of background worker slots"),
+				 errhint("You might need to increase \"%s\".", "max_worker_processes")));
+}
+
+/*
+ * Update the WAL level value on shared memory.
+ *
+ * If notify_all is true, we order all processes to update their local
+ * WAL-logging information cache using global signal barriers, and wait
+ * for the complete.
+ */
+static void
+update_wal_level(WalLevel new_wal_level, bool notify_all)
+{
+	LWLockAcquire(WalLevelControlLock, LW_EXCLUSIVE);
+	WalLevelCtl->current_wal_level = new_wal_level;
+	LWLockRelease(WalLevelControlLock);
+
+	if (notify_all)
+	{
+		WaitForProcSignalBarrier(
+								 EmitProcSignalBarrier(PROCSIGNAL_BARRIER_UPDATE_WAL_LOGGING_STATE));
+	}
+}
+
+/*
+ * Write XLOG_WAL_LEVEL_CHANGE record with the given WAL level.
+ */
+static void
+write_wal_level_change(WalLevel new_wal_level)
+{
+	XLogBeginInsert();
+	XLogRegisterData((char *) (&new_wal_level), sizeof(int));
+	XLogInsert(RM_XLOG_ID, XLOG_WAL_LEVEL_CHANGE);
+}
+
+/*
+ * Callback function for increasing WAL level to 'replica' from 'minimal'.
+ */
+static void
+increase_to_replica_action(void)
+{
+	/*
+	 * We create a checkpoint to increase wal_level from 'minimal' so that we
+	 * can restart from there in case of a server crash.
+	 */
+	RequestCheckpoint(CHECKPOINT_IMMEDIATE | CHECKPOINT_FORCE | CHECKPOINT_WAIT);
+}
+
+/*
+ * Function to increase WAL level from 'minimal' to 'replica' or from 'replica'
+ * to 'logical'.
+ *
+ * 'target_level' is the destination WAL level, which must be 'replica' or
+ * 'logical'. 'next_level' is the intermediate level for the transition, which
+ * must be STANDBY_INFO_LOGGING or LOGICAL_INFO_LOGGING.
+ */
+static void
+increase_wal_level(WalLevel next_level, WalLevel target_level,
+				   wal_level_increase_action_cb action_cb)
+
+{
+	Assert(next_level == WAL_LEVEL_STANDBY_INFO_LOGGING ||
+		   next_level == WAL_LEVEL_LOGICAL_INFO_LOGGING);
+	Assert(target_level == WAL_LEVEL_REPLICA ||
+		   target_level == WAL_LEVEL_LOGICAL);
+
+	/*
+	 * Increase to the 'next_level' so that all process can start to write WAL
+	 * records with information required by the 'target_level'. We order all
+	 * process to reflect this change.
+	 */
+	update_wal_level(next_level, true);
+
+	/* Invoke the callback function, if specified */
+	if (action_cb)
+		action_cb();
+
+	START_CRIT_SECTION();
+
+	/*
+	 * Now increase to the 'target_level'.
+	 *
+	 * It's safe to take increase the WAL level, even when not strictly
+	 * required. So first set it true and then write the WAL record.
+	 */
+	update_wal_level(target_level, false);
+	write_wal_level_change(target_level);
+
+	END_CRIT_SECTION();
+
+	ereport(LOG,
+			(errmsg("wal_level has been increased to \"%s\"",
+					get_wal_level_string(target_level))));
+}
+
+/*
+ * Callback function for decreasing WAL level to 'logical' from 'replica'.
+ */
+static void
+decrease_to_replica_action(void)
+{
+	/*
+	 * Invalidate all logical replication slots, terminating processes doing
+	 * logical decoding (and logical replication) too.
+	 */
+	InvalidateObsoleteReplicationSlots(RS_INVAL_WAL_LEVEL,
+									   0, InvalidOid,
+									   InvalidTransactionId);
+}
+
+/*
+ * Callback function for decreasing WAL level to 'replica' from 'minimal'.
+ *
+ * We terminate some functionalities that are not available at 'minimal' such
+ * as WAL archival and log shipping. The WAL level has already been decreased
+ * to WAL_LEVEL_STANDBY_INFO_LOGGING, we don't need to deal with the case
+ * where these aux processes are concurrently launched.
+ */
+static void
+decrease_to_minimal_action(void)
+{
+	/* shutdown archiver */
+	PgArchShutdown();
+
+	/* shutdown wal senders */
+	WalSndTerminate();
+	WalSndWaitStopping();
+
+	/* wait for currently running backups to finish */
+	wait_for_backup_finish();
+}
+
+/*
+ * Function to increase WAL level from 'logical' to 'replica' or from 'replica'
+ * to 'minimal'.
+ *
+ * 'target_level' is the destination WAL level, which must be 'replica' or
+ * 'minimal'. 'next_level' is the intermediate level for the transition, which
+ * must be STANDBY_INFO_LOGGING or LOGICAL_INFO_LOGGING.
+ */
+static void
+decrease_wal_level(WalLevel next_level, WalLevel target_level,
+				   wal_level_decrease_action_cb action_cb)
+{
+	Assert(next_level == WAL_LEVEL_STANDBY_INFO_LOGGING ||
+		   next_level == WAL_LEVEL_LOGICAL_INFO_LOGGING);
+	Assert(target_level == WAL_LEVEL_REPLICA ||
+		   target_level == WAL_LEVEL_MINIMAL);
+
+	/*
+	 * Decrease to the 'next_level' to prevent functionality that require the
+	 * current WAL level from being newly started while not affecting the
+	 * information contained in WAL records.
+	 */
+	update_wal_level(next_level, true);
+
+	/* Invoke the callback function, if specified */
+	if (action_cb)
+		action_cb();
+
+	START_CRIT_SECTION();
+
+	/*
+	 * Now increase to the 'target_level'.
+	 *
+	 * We must not decreasing WAL level before writing the WAL records,
+	 * because otherwise we would end up having insufficient information in
+	 * WAL records. Therefore, unlike increasing the WAL level, we write the
+	 * WAL recordand then set the shared WAL level.
+	 */
+	write_wal_level_change(target_level);
+	update_wal_level(target_level, false);
+
+	END_CRIT_SECTION();
+
+	/* Order all processed to disable logical information WAL-logging */
+	WaitForProcSignalBarrier(EmitProcSignalBarrier(PROCSIGNAL_BARRIER_UPDATE_WAL_LOGGING_STATE));
+
+	ereport(LOG,
+			(errmsg("wal_level has been decreased to \"%s\"",
+					get_wal_level_string(target_level))));
+
+}
+
+static void
+wal_level_control_worker_shutdown(int code, Datum arg)
+{
+	WalLevelCtl->worker_pid = InvalidPid;
+}
+
+void
+WalLevelCtlWorkerMain(Datum main_arg)
+{
+	WalLevel	current;
+	WalLevel	target;
+
+	ereport(LOG,
+			(errmsg("wal_level control worker started")));
+
+	LWLockAcquire(WalLevelControlLock, LW_EXCLUSIVE);
+
+	if (WalLevelCtl->worker_pid != InvalidPid)
+	{
+		LWLockRelease(WalLevelControlLock);
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 (errmsg("wal level worker is already running, cannot start"))));
+	}
+
+	if (WalLevelCtl->current_wal_level == WalLevelCtl->target_wal_level)
+	{
+		LWLockRelease(WalLevelControlLock);
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 (errmsg("no need to change wal_level, exit"))));
+	}
+
+	before_shmem_exit(wal_level_control_worker_shutdown, (Datum) 0);
+	WalLevelCtl->worker_pid = MyProcPid;
+
+	LWLockRelease(WalLevelControlLock);
+
+	pqsignal(SIGHUP, SignalHandlerForConfigReload);
+	pqsignal(SIGTERM, die);
+	BackgroundWorkerUnblockSignals();
+
+	current = WalLevelCtl->current_wal_level;
+	target = WalLevelCtl->target_wal_level;
+
+	/* The target WAL level must not be intermediate levels */
+	Assert(target == WAL_LEVEL_MINIMAL ||
+		   target == WAL_LEVEL_REPLICA ||
+		   target == WAL_LEVEL_LOGICAL);
+
+	if (current == target)
+		proc_exit(0);
+
+	ereport(LOG,
+			(errmsg("changing wal_level from \"%s\" to \"%s\"",
+					get_wal_level_string(current), get_wal_level_string(target))));
+
+	if (target > current)
+	{
+		/* We're increasing the WAL level one by one */
+
+		if (current == WAL_LEVEL_MINIMAL ||
+			current == WAL_LEVEL_STANDBY_INFO_LOGGING)
+		{
+			/* increasing to 'replica' */
+			increase_wal_level(WAL_LEVEL_STANDBY_INFO_LOGGING,
+							   WAL_LEVEL_REPLICA,
+							   increase_to_replica_action);
+		}
+
+		if (target == WAL_LEVEL_LOGICAL)
+		{
+			/* increasing to 'logical' */
+			increase_wal_level(WAL_LEVEL_LOGICAL_INFO_LOGGING,
+							   WAL_LEVEL_LOGICAL,
+							   NULL);
+		}
+	}
+	else
+	{
+		/* We're decreasing the WAL level one by one */
+
+		if (current == WAL_LEVEL_LOGICAL)
+		{
+			/* decreasing to 'replica' */
+			decrease_wal_level(WAL_LEVEL_LOGICAL_INFO_LOGGING,
+							   WAL_LEVEL_REPLICA,
+							   decrease_to_replica_action);
+		}
+
+		if (target == WAL_LEVEL_MINIMAL)
+		{
+			/* decreasing to 'replica' */
+			decrease_wal_level(WAL_LEVEL_STANDBY_INFO_LOGGING,
+							   WAL_LEVEL_MINIMAL,
+							   decrease_to_minimal_action);
+		}
+	}
+
+	ereport(LOG,
+			(errmsg("successfully made wal_level \"%s\" effective",
+					get_wal_level_string(target))));
+
+	proc_exit(0);
+}
+
+/*
+ * Find a string representation of wal_level.
+ *
+ * This function doesn't support deprecated wal_level values such
+ * as 'archive'.
+ */
+static const char *
+get_wal_level_string(WalLevel level)
+{
+	switch (level)
+	{
+		case WAL_LEVEL_MINIMAL:
+			return "minimal";
+		case WAL_LEVEL_STANDBY_INFO_LOGGING:
+			return "minimal-standby_info_logging";
+		case WAL_LEVEL_REPLICA:
+			return "replica";
+		case WAL_LEVEL_LOGICAL_INFO_LOGGING:
+			return "replica-logical_info_logging";
+		case WAL_LEVEL_LOGICAL:
+			return "logical";
+	}
+
+	return "???";
+}
+
+const char *
+show_wal_level(void)
+{
+	return get_wal_level_string(GetActiveWalLevel());
+}
+
+bool
+check_wal_level(int *newval, void **extra, GucSource source)
+{
+	/* Just accept the value when restoring state in a parallel worker */
+	if (InitializingParallelWorker)
+		return true;
+
+	if (!WalLevelInitialized)
+		return true;
+
+	if (wal_level != *newval)
+	{
+		if (RecoveryInProgress())
+		{
+			GUC_check_errmsg("cannot change \"wal_level\" during recovery");
+			return false;
+		}
+
+		if (WalLevelCtl->current_wal_level != WalLevelCtl->target_wal_level)
+		{
+			GUC_check_errmsg("cannot change \"wal_level\" while changing level is in-progress");
+			GUC_check_errdetail("\"wal_level\" is being changed to \"%s\"",
+								get_wal_level_string(WalLevelCtl->target_wal_level));
+			return false;
+		}
+	}
+
+	return true;
+}
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 473de6710d7..bc189570141 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -1122,6 +1122,11 @@ validateRecoveryParameters(void)
 			ereport(WARNING,
 					(errmsg("specified neither \"primary_conninfo\" nor \"restore_command\""),
 					 errhint("The database server will regularly poll the pg_wal subdirectory to check for files placed there.")));
+
+		if (wal_level == WAL_LEVEL_MINIMAL)
+			ereport(FATAL,
+					(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+					 errmsg("cannot enter standby mode with \"wal_level=minimal\"")));
 	}
 	else
 	{
diff --git a/src/backend/commands/publicationcmds.c b/src/backend/commands/publicationcmds.c
index 951ffabb656..b311ecaf2d2 100644
--- a/src/backend/commands/publicationcmds.c
+++ b/src/backend/commands/publicationcmds.c
@@ -900,7 +900,7 @@ CreatePublication(ParseState *pstate, CreatePublicationStmt *stmt)
 
 	InvokeObjectPostCreateHook(PublicationRelationId, puboid, 0);
 
-	if (wal_level != WAL_LEVEL_LOGICAL)
+	if (!XLogLogicalInfoActive())
 		ereport(WARNING,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 				 errmsg("\"wal_level\" is insufficient to publish logical changes"),
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index b288915cec8..efb0ff18e8a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -13,6 +13,7 @@
 #include "postgres.h"
 
 #include "access/parallel.h"
+#include "access/xloglevelworker.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
 #include "pgstat.h"
@@ -132,6 +133,9 @@ static const struct
 	},
 	{
 		"TablesyncWorkerMain", TablesyncWorkerMain
+	},
+	{
+		"WalLevelCtlWorkerMain", WalLevelCtlWorkerMain
 	}
 };
 
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index b94f9cdff21..498e4b0190b 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -42,6 +42,7 @@
 #include "access/xlog.h"
 #include "access/xlog_internal.h"
 #include "access/xlogrecovery.h"
+#include "access/xloglevelworker.h"
 #include "libpq/pqsignal.h"
 #include "miscadmin.h"
 #include "pgstat.h"
@@ -1377,6 +1378,12 @@ UpdateSharedMemoryConfig(void)
 	 */
 	UpdateFullPageWrites();
 
+	/*
+	 * If wal_level has been changed by SIGHUP, delegate changing wal_level
+	 * value to the wal level control worker.
+	 */
+	UpdateWalLevel();
+
 	elog(DEBUG2, "checkpointer updated shared memory configuration values");
 }
 
diff --git a/src/backend/postmaster/pgarch.c b/src/backend/postmaster/pgarch.c
index 12ee815a626..d3001c6ebd3 100644
--- a/src/backend/postmaster/pgarch.c
+++ b/src/backend/postmaster/pgarch.c
@@ -291,6 +291,32 @@ PgArchWakeup(void)
 		SetLatch(&ProcGlobal->allProcs[arch_pgprocno].procLatch);
 }
 
+void
+PgArchShutdown(void)
+{
+	int			arch_pgprocno = PgArch->pgprocno;
+
+	if (arch_pgprocno == INVALID_PROC_NUMBER)
+		return;
+
+	/* terminate archiver */
+	kill(GetPGProcByNumber(arch_pgprocno)->pid, SIGUSR2);
+
+	/* and wait for it to exit */
+	for (;;)
+	{
+		CHECK_FOR_INTERRUPTS();
+
+		/* is it gone? */
+		if (arch_pgprocno == INVALID_PROC_NUMBER)
+			break;
+
+		WaitLatch(MyLatch,
+				  WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
+				  100L, WAIT_EVENT_ARCHIVER_SHUTDOWN);
+		ResetLatch(MyLatch);
+	}
+}
 
 /* SIGUSR2 signal handler for archiver process */
 static void
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index bb22b13adef..f327e0aef6e 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -835,15 +835,6 @@ PostmasterMain(int argc, char *argv[])
 					 MaxConnections);
 		ExitPostmaster(1);
 	}
-	if (XLogArchiveMode > ARCHIVE_MODE_OFF && wal_level == WAL_LEVEL_MINIMAL)
-		ereport(ERROR,
-				(errmsg("WAL archival cannot be enabled when \"wal_level\" is \"minimal\"")));
-	if (max_wal_senders > 0 && wal_level == WAL_LEVEL_MINIMAL)
-		ereport(ERROR,
-				(errmsg("WAL streaming (\"max_wal_senders\" > 0) requires \"wal_level\" to be \"replica\" or \"logical\"")));
-	if (summarize_wal && wal_level == WAL_LEVEL_MINIMAL)
-		ereport(ERROR,
-				(errmsg("WAL cannot be summarized when \"wal_level\" is \"minimal\"")));
 
 	/*
 	 * Other one-time internal sanity checks can go here, if they are fast.
@@ -1030,6 +1021,12 @@ PostmasterMain(int argc, char *argv[])
 	RemovePgTempFilesInDir(PG_TEMP_FILES_DIR, true, false);
 #endif
 
+	/*
+	 * Initialize logical info logging and logical decoding state. We must do
+	 * this after initializing GUCs as it reflects wal_level setting.
+	 */
+	InitializeWalLevelCtl();
+
 	/*
 	 * Forcibly remove the files signaling a standby promotion request.
 	 * Otherwise, the existence of those files triggers a promotion too early,
@@ -3279,7 +3276,7 @@ LaunchMissingBackgroundProcesses(void)
 	 * If WAL archiving is enabled always, we are allowed to start archiver
 	 * even during recovery.
 	 */
-	if (PgArchPMChild == NULL &&
+	if (GetActiveWalLevel() >= WAL_LEVEL_REPLICA && PgArchPMChild == NULL &&
 		((XLogArchivingActive() && pmState == PM_RUN) ||
 		 (XLogArchivingAlways() && (pmState == PM_RECOVERY || pmState == PM_HOT_STANDBY))) &&
 		PgArchCanRestart())
@@ -3325,7 +3322,8 @@ LaunchMissingBackgroundProcesses(void)
 	}
 
 	/* If we need to start a WAL summarizer, try to do that now */
-	if (summarize_wal && WalSummarizerPMChild == NULL &&
+	if (summarize_wal && GetActiveWalLevel() >= WAL_LEVEL_REPLICA &&
+		WalSummarizerPMChild == NULL &&
 		(pmState == PM_RUN || pmState == PM_HOT_STANDBY) &&
 		Shutdown <= SmartShutdown)
 		WalSummarizerPMChild = StartChildProcess(B_WAL_SUMMARIZER);
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 0bff0f10652..d41db7b983f 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -178,6 +178,18 @@ xlog_decode(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 				}
 				break;
 			}
+		case XLOG_WAL_LEVEL_CHANGE:
+			{
+				int			new_wal_level;
+
+				memcpy(&new_wal_level, XLogRecGetData(buf->record),
+					   sizeof(new_wal_level));
+
+				if (new_wal_level < WAL_LEVEL_LOGICAL)
+					ereport(ERROR,
+							(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+							 errmsg("logical decoding on standby requires \"wal_level\" >= \"logical\" on the primary")));
+			}
 		case XLOG_NOOP:
 		case XLOG_NEXTOID:
 		case XLOG_SWITCH:
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 8ea846bfc3b..8679b2d0ef7 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -115,7 +115,7 @@ CheckLogicalDecodingRequirements(void)
 	 * needs the same check.
 	 */
 
-	if (wal_level < WAL_LEVEL_LOGICAL)
+	if (!LogicalDecodingEnabled())
 		ereport(ERROR,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 				 errmsg("logical decoding requires \"wal_level\" >= \"logical\"")));
diff --git a/src/backend/replication/logical/slotsync.c b/src/backend/replication/logical/slotsync.c
index 987857b9491..3cae6ab7d41 100644
--- a/src/backend/replication/logical/slotsync.c
+++ b/src/backend/replication/logical/slotsync.c
@@ -1038,14 +1038,14 @@ ValidateSlotSyncParams(int elevel)
 {
 	/*
 	 * Logical slot sync/creation requires wal_level >= logical.
-	 *
-	 * Since altering the wal_level requires a server restart, so error out in
-	 * this case regardless of elevel provided by caller.
 	 */
-	if (wal_level < WAL_LEVEL_LOGICAL)
-		ereport(ERROR,
+	if (!LogicalDecodingEnabled())
+	{
+		ereport(elevel,
 				errcode(ERRCODE_INVALID_PARAMETER_VALUE),
 				errmsg("replication slot synchronization requires \"wal_level\" >= \"logical\""));
+		return false;
+	}
 
 	/*
 	 * A physical replication slot(primary_slot_name) is required on the
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index fe5acd8b1fc..b092008ce56 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -1403,7 +1403,7 @@ CheckSlotRequirements(void)
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 				 errmsg("replication slots can only be used if \"max_replication_slots\" > 0")));
 
-	if (wal_level < WAL_LEVEL_REPLICA)
+	if (GetActiveWalLevel() < WAL_LEVEL_REPLICA)
 		ereport(ERROR,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 				 errmsg("replication slots can only be used if \"wal_level\" >= \"replica\"")));
@@ -2360,13 +2360,13 @@ RestoreSlotFromDisk(const char *name)
 	 * NB: Changing the requirements here also requires adapting
 	 * CheckSlotRequirements() and CheckLogicalDecodingRequirements().
 	 */
-	if (cp.slotdata.database != InvalidOid && wal_level < WAL_LEVEL_LOGICAL)
+	if (cp.slotdata.database != InvalidOid && GetActiveWalLevel() < WAL_LEVEL_LOGICAL)
 		ereport(FATAL,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 				 errmsg("logical replication slot \"%s\" exists, but \"wal_level\" < \"logical\"",
 						NameStr(cp.slotdata.name)),
 				 errhint("Change \"wal_level\" to be \"logical\" or higher.")));
-	else if (wal_level < WAL_LEVEL_REPLICA)
+	else if (GetActiveWalLevel() < WAL_LEVEL_REPLICA)
 		ereport(FATAL,
 				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
 				 errmsg("physical replication slot \"%s\" exists, but \"wal_level\" < \"replica\"",
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 446d10c1a7d..9d0b33cf833 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -277,6 +277,11 @@ static void WalSndSegmentOpen(XLogReaderState *state, XLogSegNo nextSegNo,
 void
 InitWalSender(void)
 {
+	if (GetActiveWalLevel() < WAL_LEVEL_REPLICA)
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("WAL senders require \"wal_level\" to be \"replica\" or \"logical\"")));
+
 	am_cascading_walsender = RecoveryInProgress();
 
 	/* Create a per-walsender data structure in shared memory */
@@ -3733,6 +3738,30 @@ WalSndInitStopping(void)
 	}
 }
 
+/*
+ * Terminate all running walsenders. Unlike shutting down them using
+ * WalSndInitStopping, this function can be used to terminate walsenders
+ * even while normal backends are generating WAL records.
+ */
+void
+WalSndTerminate(void)
+{
+	for (int i = 0; i < max_wal_senders; i++)
+	{
+		WalSnd	   *walsnd = &WalSndCtl->walsnds[i];
+		pid_t		pid;
+
+		SpinLockAcquire(&walsnd->mutex);
+		pid = walsnd->pid;
+		SpinLockRelease(&walsnd->mutex);
+
+		if (pid == 0)
+			continue;
+
+		kill(pid, SIGUSR2);
+	}
+}
+
 /*
  * Wait that all the WAL senders have quit or reached the stopping state. This
  * is used by the checkpointer to control when the shutdown checkpoint can
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index 174eed70367..86ad378f169 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -148,6 +148,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, WaitEventCustomShmemSize());
 	size = add_size(size, InjectionPointShmemSize());
 	size = add_size(size, SlotSyncShmemSize());
+	size = add_size(size, WalLevelCtlShmemSize());
 
 	/* include additional requested shmem from preload libraries */
 	size = add_size(size, total_addin_request);
@@ -330,6 +331,7 @@ CreateOrAttachShmemStructs(void)
 	PgArchShmemInit();
 	ApplyLauncherShmemInit();
 	SlotSyncShmemInit();
+	WalLevelCtlShmemInit();
 
 	/*
 	 * Set up other modules that need some shared memory space
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 7401b6e625e..b930036d3a3 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -18,6 +18,7 @@
 #include <unistd.h>
 
 #include "access/parallel.h"
+#include "access/xlog.h"
 #include "commands/async.h"
 #include "miscadmin.h"
 #include "pgstat.h"
@@ -573,6 +574,9 @@ ProcessProcSignalBarrier(void)
 					case PROCSIGNAL_BARRIER_SMGRRELEASE:
 						processed = ProcessBarrierSmgrRelease();
 						break;
+					case PROCSIGNAL_BARRIER_UPDATE_WAL_LOGGING_STATE:
+						processed = ProcessBarrierUpdateWalLoggingState();
+						break;
 				}
 
 				/*
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 2039062554d..0ddc834ebaa 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -498,7 +498,7 @@ ResolveRecoveryConflictWithSnapshot(TransactionId snapshotConflictHorizon,
 	 * seems OK, given that this kind of conflict should not normally be
 	 * reached, e.g. due to using a physical replication slot.
 	 */
-	if (wal_level >= WAL_LEVEL_LOGICAL && isCatalogRel)
+	if (GetActiveWalLevel() >= WAL_LEVEL_LOGICAL && isCatalogRel)
 		InvalidateObsoleteReplicationSlots(RS_INVAL_HORIZON, 0, locator.dbOid,
 										   snapshotConflictHorizon);
 }
@@ -1313,13 +1313,13 @@ LogStandbySnapshot(void)
 	 * record. Fortunately this routine isn't executed frequently, and it's
 	 * only a shared lock.
 	 */
-	if (wal_level < WAL_LEVEL_LOGICAL)
+	if (!XLogLogicalInfoActive())
 		LWLockRelease(ProcArrayLock);
 
 	recptr = LogCurrentRunningXacts(running);
 
 	/* Release lock if we kept it longer ... */
-	if (wal_level >= WAL_LEVEL_LOGICAL)
+	if (XLogLogicalInfoActive())
 		LWLockRelease(ProcArrayLock);
 
 	/* GetRunningTransactionData() acquired XidGenLock, we must release it */
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index e199f071628..b81cab14e60 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -52,6 +52,7 @@
 Section: ClassName - WaitEventActivity
 
 ARCHIVER_MAIN	"Waiting in main loop of archiver process."
+ARCHIVER_SHUTDOWN	"Waiting for archiver process to be terminated."
 AUTOVACUUM_MAIN	"Waiting in main loop of autovacuum launcher process."
 BGWRITER_HIBERNATE	"Waiting in background writer process, hibernating."
 BGWRITER_MAIN	"Waiting in main loop of background writer process."
@@ -106,6 +107,7 @@ APPEND_READY	"Waiting for subplan nodes of an <literal>Append</literal> plan nod
 ARCHIVE_CLEANUP_COMMAND	"Waiting for <xref linkend="guc-archive-cleanup-command"/> to complete."
 ARCHIVE_COMMAND	"Waiting for <xref linkend="guc-archive-command"/> to complete."
 BACKEND_TERMINATION	"Waiting for the termination of another backend."
+BACKUP_TERMINATION	"Waiting for all running backups to complete."
 BACKUP_WAIT_WAL_ARCHIVE	"Waiting for WAL files required for a backup to be successfully archived."
 BGWORKER_SHUTDOWN	"Waiting for background worker to shut down."
 BGWORKER_STARTUP	"Waiting for background worker to start up."
@@ -346,6 +348,7 @@ WALSummarizer	"Waiting to read or update WAL summarization state."
 DSMRegistry	"Waiting to read or update the dynamic shared memory registry."
 InjectionPoint	"Waiting to read or update information related to injection points."
 SerialControl	"Waiting to read or update shared <filename>pg_serial</filename> state."
+WalLevelControl	"Waiting to read or update wal_level control state."
 
 #
 # END OF PREDEFINED LWLOCKS (DO NOT CHANGE THIS LINE)
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index 01bb6a410cb..26e0c612304 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -646,6 +646,9 @@ BaseInit(void)
 	/* Initialize lock manager's local structs */
 	InitLockManagerAccess();
 
+	/* Initialize WAL-logging state */
+	InitWalLoggingState();
+
 	/*
 	 * Initialize replication slots after pgstat. The exit hook might need to
 	 * drop ephemeral slots, which in turn triggers stats reporting.
diff --git a/src/backend/utils/misc/guc_tables.c b/src/backend/utils/misc/guc_tables.c
index ce7534d4d23..b23ca38dcfb 100644
--- a/src/backend/utils/misc/guc_tables.c
+++ b/src/backend/utils/misc/guc_tables.c
@@ -5072,13 +5072,13 @@ struct config_enum ConfigureNamesEnum[] =
 	},
 
 	{
-		{"wal_level", PGC_POSTMASTER, WAL_SETTINGS,
+		{"wal_level", PGC_SIGHUP, WAL_SETTINGS,
 			gettext_noop("Sets the level of information written to the WAL."),
 			NULL
 		},
 		&wal_level,
 		WAL_LEVEL_REPLICA, wal_level_options,
-		NULL, NULL, NULL
+		check_wal_level, NULL, show_wal_level
 	},
 
 	{
diff --git a/src/bin/pg_controldata/pg_controldata.c b/src/bin/pg_controldata/pg_controldata.c
index cf11ab3f2ee..61a5593869d 100644
--- a/src/bin/pg_controldata/pg_controldata.c
+++ b/src/bin/pg_controldata/pg_controldata.c
@@ -76,8 +76,12 @@ wal_level_str(WalLevel wal_level)
 	{
 		case WAL_LEVEL_MINIMAL:
 			return "minimal";
+		case WAL_LEVEL_STANDBY_INFO_LOGGING:
+			return "minimal-standby_info_logging";
 		case WAL_LEVEL_REPLICA:
 			return "replica";
+		case WAL_LEVEL_LOGICAL_INFO_LOGGING:
+			return "replica-logical_info_logging";
 		case WAL_LEVEL_LOGICAL:
 			return "logical";
 	}
diff --git a/src/include/access/xlog.h b/src/include/access/xlog.h
index 4411c1468ac..bb2a8353ac0 100644
--- a/src/include/access/xlog.h
+++ b/src/include/access/xlog.h
@@ -71,10 +71,38 @@ extern PGDLLIMPORT int XLogArchiveMode;
 /* WAL levels */
 typedef enum WalLevel
 {
+	/*
+	 * This WAL level corresponds to 'minimal', where we don't WAL-logging for
+	 * neither archival nor log-shipping.
+	 */
 	WAL_LEVEL_MINIMAL = 0,
+
+	/*
+	 * In this level, we enable WAL-logging information required only for
+	 * archival, log-shipping, and hot standby. However, these functionalities
+	 * are not available with this level.
+	 */
+	WAL_LEVEL_STANDBY_INFO_LOGGING,
+
+	/*
+	 * This WAL level corresponds to 'replica', where we allow archival, and,
+	 * log-shipping, and hot standby.
+	 */
 	WAL_LEVEL_REPLICA,
+
+	/*
+	 * In this level, we enable WAL-logging information required for logical
+	 * decoding. However, logical decoding is not available with this level.
+	 */
+	WAL_LEVEL_LOGICAL_INFO_LOGGING,
+
+	/*
+	 * This WAL level corresponds to 'logical', where we allow logical
+	 * decoding.
+	 */
 	WAL_LEVEL_LOGICAL,
 } WalLevel;
+extern bool WalLevelInitialized;
 
 /* Compression algorithms for WAL */
 typedef enum WalCompression
@@ -97,16 +125,16 @@ extern PGDLLIMPORT int wal_level;
 
 /* Is WAL archiving enabled (always or only while server is running normally)? */
 #define XLogArchivingActive() \
-	(AssertMacro(XLogArchiveMode == ARCHIVE_MODE_OFF || wal_level >= WAL_LEVEL_REPLICA), XLogArchiveMode > ARCHIVE_MODE_OFF)
+	(AssertMacro(XLogArchiveMode == ARCHIVE_MODE_OFF || GetActiveWalLevel() >= WAL_LEVEL_REPLICA), XLogArchiveMode > ARCHIVE_MODE_OFF)
 /* Is WAL archiving enabled always (even during recovery)? */
 #define XLogArchivingAlways() \
-	(AssertMacro(XLogArchiveMode == ARCHIVE_MODE_OFF || wal_level >= WAL_LEVEL_REPLICA), XLogArchiveMode == ARCHIVE_MODE_ALWAYS)
+	(AssertMacro(XLogArchiveMode == ARCHIVE_MODE_OFF || GetActiveWalLevel() >= WAL_LEVEL_REPLICA), XLogArchiveMode == ARCHIVE_MODE_ALWAYS)
 
 /*
  * Is WAL-logging necessary for archival or log-shipping, or can we skip
  * WAL-logging if we fsync() the data before committing instead?
  */
-#define XLogIsNeeded() (wal_level >= WAL_LEVEL_REPLICA)
+#define XLogIsNeeded() (StandbyInfoLoggingEnabled())
 
 /*
  * Is a full-page image needed for hint bit updates?
@@ -120,10 +148,10 @@ extern PGDLLIMPORT int wal_level;
 #define XLogHintBitIsNeeded() (DataChecksumsEnabled() || wal_log_hints)
 
 /* Do we need to WAL-log information required only for Hot Standby and logical replication? */
-#define XLogStandbyInfoActive() (wal_level >= WAL_LEVEL_REPLICA)
+#define XLogStandbyInfoActive() (StandbyInfoLoggingEnabled())
 
 /* Do we need to WAL-log information required only for logical replication? */
-#define XLogLogicalInfoActive() (wal_level >= WAL_LEVEL_LOGICAL)
+#define XLogLogicalInfoActive() (LogicalInfoLoggingEnabled())
 
 #ifdef WAL_DEBUG
 extern PGDLLIMPORT bool XLOG_DEBUG;
@@ -270,6 +298,21 @@ extern void SetInstallXLogFileSegmentActive(void);
 extern bool IsInstallXLogFileSegmentActive(void);
 extern void XLogShutdownWalRcv(void);
 
+/*
+ * Routines used by xloglevelworker.c to control WAL-logging information.
+ */
+extern Size WalLevelCtlShmemSize(void);
+extern void WalLevelCtlShmemInit(void);
+extern void InitializeWalLevelCtl(void);
+extern void InitWalLoggingState(void);
+extern bool ProcessBarrierUpdateWalLoggingState(void);
+extern bool LogicalInfoLoggingEnabled(void);
+extern bool StandbyInfoLoggingEnabled(void);
+extern bool LogicalDecodingEnabled(void);
+extern int	GetActiveWalLevel(void);
+extern void UpdateWalLevelAfterRecovery(WalLevel level);
+extern void UpdateWalLevel(void);
+
 /*
  * Routines to start, stop, and get status of a base backup.
  */
@@ -297,6 +340,7 @@ extern void do_pg_backup_stop(BackupState *state, bool waitforarchive);
 extern void do_pg_abort_backup(int code, Datum arg);
 extern void register_persistent_abort_backup_handler(void);
 extern SessionBackupState get_backup_status(void);
+extern void wait_for_backup_finish(void);
 
 /* File path names (all relative to $PGDATA) */
 #define RECOVERY_SIGNAL_FILE	"recovery.signal"
diff --git a/src/include/access/xloglevelworker.h b/src/include/access/xloglevelworker.h
new file mode 100644
index 00000000000..1082b3994c9
--- /dev/null
+++ b/src/include/access/xloglevelworker.h
@@ -0,0 +1,18 @@
+/*-------------------------------------------------------------------------
+ *
+ * xloglevelworker.h
+ *
+ * Portions Copyright (c) 1996-2025, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/include/access/xloglevelworker.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef XLOGLEVELWORKER_H
+#define XLOGLEVELWORKER_H
+
+extern void WalLevelCtlWorkerLaunchIfNecessary(void);
+extern void WalLevelCtlWorkerMain(Datum main_arg);
+
+#endif
diff --git a/src/include/catalog/pg_control.h b/src/include/catalog/pg_control.h
index 3797f25b306..39a14c67f6a 100644
--- a/src/include/catalog/pg_control.h
+++ b/src/include/catalog/pg_control.h
@@ -80,6 +80,7 @@ typedef struct CheckPoint
 /* 0xC0 is used in Postgres 9.5-11 */
 #define XLOG_OVERWRITE_CONTRECORD		0xD0
 #define XLOG_CHECKPOINT_REDO			0xE0
+#define XLOG_WAL_LEVEL_CHANGE			0xF0
 
 
 /*
diff --git a/src/include/postmaster/pgarch.h b/src/include/postmaster/pgarch.h
index 8fc6bfeec1b..73f2d13223d 100644
--- a/src/include/postmaster/pgarch.h
+++ b/src/include/postmaster/pgarch.h
@@ -31,6 +31,7 @@ extern void PgArchShmemInit(void);
 extern bool PgArchCanRestart(void);
 extern void PgArchiverMain(char *startup_data, size_t startup_data_len) pg_attribute_noreturn();
 extern void PgArchWakeup(void);
+extern void PgArchShutdown(void);
 extern void PgArchForceDirScan(void);
 
 #endif							/* _PGARCH_H */
diff --git a/src/include/replication/walsender.h b/src/include/replication/walsender.h
index c3e8e191339..d6d426915e9 100644
--- a/src/include/replication/walsender.h
+++ b/src/include/replication/walsender.h
@@ -44,6 +44,7 @@ extern void WalSndSignals(void);
 extern Size WalSndShmemSize(void);
 extern void WalSndShmemInit(void);
 extern void WalSndWakeup(bool physical, bool logical);
+extern void WalSndTerminate(void);
 extern void WalSndInitStopping(void);
 extern void WalSndWaitStopping(void);
 extern void HandleWalSndInitStopping(void);
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index cf565452382..979c8a4f010 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -83,3 +83,4 @@ PG_LWLOCK(49, WALSummarizer)
 PG_LWLOCK(50, DSMRegistry)
 PG_LWLOCK(51, InjectionPoint)
 PG_LWLOCK(52, SerialControl)
+PG_LWLOCK(53, WalLevelControl)
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index 022fd8ed933..fc5b238ec20 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -54,6 +54,8 @@ typedef enum
 typedef enum
 {
 	PROCSIGNAL_BARRIER_SMGRRELEASE, /* ask smgr to close files */
+	PROCSIGNAL_BARRIER_UPDATE_WAL_LOGGING_STATE,	/* ask to update xlog info
+													 * state */
 } ProcSignalBarrierType;
 
 /*
diff --git a/src/include/utils/guc_hooks.h b/src/include/utils/guc_hooks.h
index 87999218d68..b8d36e0a9b8 100644
--- a/src/include/utils/guc_hooks.h
+++ b/src/include/utils/guc_hooks.h
@@ -169,6 +169,8 @@ extern bool check_wal_buffers(int *newval, void **extra, GucSource source);
 extern bool check_wal_consistency_checking(char **newval, void **extra,
 										   GucSource source);
 extern void assign_wal_consistency_checking(const char *newval, void *extra);
+extern const char *show_wal_level(void);
+extern bool check_wal_level(int *newval, void **extra, GucSource source);
 extern bool check_wal_segment_size(int *newval, void **extra, GucSource source);
 extern void assign_wal_sync_method(int new_wal_sync_method, void *extra);
 extern bool check_synchronized_standby_slots(char **newval, void **extra,
-- 
2.43.5

