nbtree: fix read page recheck typo.
[pgsql.git] / src / backend / replication / walreceiver.c
bloba27aee63defca301f1078a9f113d27b437bd0a13
1 /*-------------------------------------------------------------------------
3 * walreceiver.c
5 * The WAL receiver process (walreceiver) is new as of Postgres 9.0. It
6 * is the process in the standby server that takes charge of receiving
7 * XLOG records from a primary server during streaming replication.
9 * When the startup process determines that it's time to start streaming,
10 * it instructs postmaster to start walreceiver. Walreceiver first connects
11 * to the primary server (it will be served by a walsender process
12 * in the primary server), and then keeps receiving XLOG records and
13 * writing them to the disk as long as the connection is alive. As XLOG
14 * records are received and flushed to disk, it updates the
15 * WalRcv->flushedUpto variable in shared memory, to inform the startup
16 * process of how far it can proceed with XLOG replay.
18 * A WAL receiver cannot directly load GUC parameters used when establishing
19 * its connection to the primary. Instead it relies on parameter values
20 * that are passed down by the startup process when streaming is requested.
21 * This applies, for example, to the replication slot and the connection
22 * string to be used for the connection with the primary.
24 * If the primary server ends streaming, but doesn't disconnect, walreceiver
25 * goes into "waiting" mode, and waits for the startup process to give new
26 * instructions. The startup process will treat that the same as
27 * disconnection, and will rescan the archive/pg_wal directory. But when the
28 * startup process wants to try streaming replication again, it will just
29 * nudge the existing walreceiver process that's waiting, instead of launching
30 * a new one.
32 * Normal termination is by SIGTERM, which instructs the walreceiver to
33 * exit(0). Emergency termination is by SIGQUIT; like any postmaster child
34 * process, the walreceiver will simply abort and exit on SIGQUIT. A close
35 * of the connection and a FATAL error are treated not as a crash but as
36 * normal operation.
38 * This file contains the server-facing parts of walreceiver. The libpq-
39 * specific parts are in the libpqwalreceiver module. It's loaded
40 * dynamically to avoid linking the server with libpq.
42 * Portions Copyright (c) 2010-2024, PostgreSQL Global Development Group
45 * IDENTIFICATION
46 * src/backend/replication/walreceiver.c
48 *-------------------------------------------------------------------------
50 #include "postgres.h"
52 #include <unistd.h>
54 #include "access/htup_details.h"
55 #include "access/timeline.h"
56 #include "access/transam.h"
57 #include "access/xlog_internal.h"
58 #include "access/xlogarchive.h"
59 #include "access/xlogrecovery.h"
60 #include "catalog/pg_authid.h"
61 #include "funcapi.h"
62 #include "libpq/pqformat.h"
63 #include "libpq/pqsignal.h"
64 #include "miscadmin.h"
65 #include "pgstat.h"
66 #include "postmaster/auxprocess.h"
67 #include "postmaster/interrupt.h"
68 #include "replication/walreceiver.h"
69 #include "replication/walsender.h"
70 #include "storage/ipc.h"
71 #include "storage/proc.h"
72 #include "storage/procarray.h"
73 #include "storage/procsignal.h"
74 #include "utils/acl.h"
75 #include "utils/builtins.h"
76 #include "utils/guc.h"
77 #include "utils/pg_lsn.h"
78 #include "utils/ps_status.h"
79 #include "utils/timestamp.h"
83 * GUC variables. (Other variables that affect walreceiver are in xlog.c
84 * because they're passed down from the startup process, for better
85 * synchronization.)
87 int wal_receiver_status_interval;
88 int wal_receiver_timeout;
89 bool hot_standby_feedback;
91 /* libpqwalreceiver connection */
92 static WalReceiverConn *wrconn = NULL;
93 WalReceiverFunctionsType *WalReceiverFunctions = NULL;
96 * These variables are used similarly to openLogFile/SegNo,
97 * but for walreceiver to write the XLOG. recvFileTLI is the TimeLineID
98 * corresponding the filename of recvFile.
100 static int recvFile = -1;
101 static TimeLineID recvFileTLI = 0;
102 static XLogSegNo recvSegNo = 0;
105 * LogstreamResult indicates the byte positions that we have already
106 * written/fsynced.
108 static struct
110 XLogRecPtr Write; /* last byte + 1 written out in the standby */
111 XLogRecPtr Flush; /* last byte + 1 flushed in the standby */
112 } LogstreamResult;
115 * Reasons to wake up and perform periodic tasks.
117 typedef enum WalRcvWakeupReason
119 WALRCV_WAKEUP_TERMINATE,
120 WALRCV_WAKEUP_PING,
121 WALRCV_WAKEUP_REPLY,
122 WALRCV_WAKEUP_HSFEEDBACK,
123 #define NUM_WALRCV_WAKEUPS (WALRCV_WAKEUP_HSFEEDBACK + 1)
124 } WalRcvWakeupReason;
127 * Wake up times for periodic tasks.
129 static TimestampTz wakeup[NUM_WALRCV_WAKEUPS];
131 static StringInfoData reply_message;
133 /* Prototypes for private functions */
134 static void WalRcvFetchTimeLineHistoryFiles(TimeLineID first, TimeLineID last);
135 static void WalRcvWaitForStartPosition(XLogRecPtr *startpoint, TimeLineID *startpointTLI);
136 static void WalRcvDie(int code, Datum arg);
137 static void XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len,
138 TimeLineID tli);
139 static void XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr,
140 TimeLineID tli);
141 static void XLogWalRcvFlush(bool dying, TimeLineID tli);
142 static void XLogWalRcvClose(XLogRecPtr recptr, TimeLineID tli);
143 static void XLogWalRcvSendReply(bool force, bool requestReply);
144 static void XLogWalRcvSendHSFeedback(bool immed);
145 static void ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime);
146 static void WalRcvComputeNextWakeup(WalRcvWakeupReason reason, TimestampTz now);
149 * Process any interrupts the walreceiver process may have received.
150 * This should be called any time the process's latch has become set.
152 * Currently, only SIGTERM is of interest. We can't just exit(1) within the
153 * SIGTERM signal handler, because the signal might arrive in the middle of
154 * some critical operation, like while we're holding a spinlock. Instead, the
155 * signal handler sets a flag variable as well as setting the process's latch.
156 * We must check the flag (by calling ProcessWalRcvInterrupts) anytime the
157 * latch has become set. Operations that could block for a long time, such as
158 * reading from a remote server, must pay attention to the latch too; see
159 * libpqrcv_PQgetResult for example.
161 void
162 ProcessWalRcvInterrupts(void)
165 * Although walreceiver interrupt handling doesn't use the same scheme as
166 * regular backends, call CHECK_FOR_INTERRUPTS() to make sure we receive
167 * any incoming signals on Win32, and also to make sure we process any
168 * barrier events.
170 CHECK_FOR_INTERRUPTS();
172 if (ShutdownRequestPending)
174 ereport(FATAL,
175 (errcode(ERRCODE_ADMIN_SHUTDOWN),
176 errmsg("terminating walreceiver process due to administrator command")));
181 /* Main entry point for walreceiver process */
182 void
183 WalReceiverMain(char *startup_data, size_t startup_data_len)
185 char conninfo[MAXCONNINFO];
186 char *tmp_conninfo;
187 char slotname[NAMEDATALEN];
188 bool is_temp_slot;
189 XLogRecPtr startpoint;
190 TimeLineID startpointTLI;
191 TimeLineID primaryTLI;
192 bool first_stream;
193 WalRcvData *walrcv;
194 TimestampTz now;
195 char *err;
196 char *sender_host = NULL;
197 int sender_port = 0;
198 char *appname;
200 Assert(startup_data_len == 0);
202 MyBackendType = B_WAL_RECEIVER;
203 AuxiliaryProcessMainCommon();
206 * WalRcv should be set up already (if we are a backend, we inherit this
207 * by fork() or EXEC_BACKEND mechanism from the postmaster).
209 walrcv = WalRcv;
210 Assert(walrcv != NULL);
213 * Mark walreceiver as running in shared memory.
215 * Do this as early as possible, so that if we fail later on, we'll set
216 * state to STOPPED. If we die before this, the startup process will keep
217 * waiting for us to start up, until it times out.
219 SpinLockAcquire(&walrcv->mutex);
220 Assert(walrcv->pid == 0);
221 switch (walrcv->walRcvState)
223 case WALRCV_STOPPING:
224 /* If we've already been requested to stop, don't start up. */
225 walrcv->walRcvState = WALRCV_STOPPED;
226 /* fall through */
228 case WALRCV_STOPPED:
229 SpinLockRelease(&walrcv->mutex);
230 ConditionVariableBroadcast(&walrcv->walRcvStoppedCV);
231 proc_exit(1);
232 break;
234 case WALRCV_STARTING:
235 /* The usual case */
236 break;
238 case WALRCV_WAITING:
239 case WALRCV_STREAMING:
240 case WALRCV_RESTARTING:
241 default:
242 /* Shouldn't happen */
243 SpinLockRelease(&walrcv->mutex);
244 elog(PANIC, "walreceiver still running according to shared memory state");
246 /* Advertise our PID so that the startup process can kill us */
247 walrcv->pid = MyProcPid;
248 walrcv->walRcvState = WALRCV_STREAMING;
250 /* Fetch information required to start streaming */
251 walrcv->ready_to_display = false;
252 strlcpy(conninfo, (char *) walrcv->conninfo, MAXCONNINFO);
253 strlcpy(slotname, (char *) walrcv->slotname, NAMEDATALEN);
254 is_temp_slot = walrcv->is_temp_slot;
255 startpoint = walrcv->receiveStart;
256 startpointTLI = walrcv->receiveStartTLI;
259 * At most one of is_temp_slot and slotname can be set; otherwise,
260 * RequestXLogStreaming messed up.
262 Assert(!is_temp_slot || (slotname[0] == '\0'));
264 /* Initialise to a sanish value */
265 now = GetCurrentTimestamp();
266 walrcv->lastMsgSendTime =
267 walrcv->lastMsgReceiptTime = walrcv->latestWalEndTime = now;
269 /* Report the latch to use to awaken this process */
270 walrcv->latch = &MyProc->procLatch;
272 SpinLockRelease(&walrcv->mutex);
274 pg_atomic_write_u64(&WalRcv->writtenUpto, 0);
276 /* Arrange to clean up at walreceiver exit */
277 on_shmem_exit(WalRcvDie, PointerGetDatum(&startpointTLI));
279 /* Properly accept or ignore signals the postmaster might send us */
280 pqsignal(SIGHUP, SignalHandlerForConfigReload); /* set flag to read config
281 * file */
282 pqsignal(SIGINT, SIG_IGN);
283 pqsignal(SIGTERM, SignalHandlerForShutdownRequest); /* request shutdown */
284 /* SIGQUIT handler was already set up by InitPostmasterChild */
285 pqsignal(SIGALRM, SIG_IGN);
286 pqsignal(SIGPIPE, SIG_IGN);
287 pqsignal(SIGUSR1, procsignal_sigusr1_handler);
288 pqsignal(SIGUSR2, SIG_IGN);
290 /* Reset some signals that are accepted by postmaster but not here */
291 pqsignal(SIGCHLD, SIG_DFL);
293 /* Load the libpq-specific functions */
294 load_file("libpqwalreceiver", false);
295 if (WalReceiverFunctions == NULL)
296 elog(ERROR, "libpqwalreceiver didn't initialize correctly");
298 /* Unblock signals (they were blocked when the postmaster forked us) */
299 sigprocmask(SIG_SETMASK, &UnBlockSig, NULL);
301 /* Establish the connection to the primary for XLOG streaming */
302 appname = cluster_name[0] ? cluster_name : "walreceiver";
303 wrconn = walrcv_connect(conninfo, true, false, false, appname, &err);
304 if (!wrconn)
305 ereport(ERROR,
306 (errcode(ERRCODE_CONNECTION_FAILURE),
307 errmsg("streaming replication receiver \"%s\" could not connect to the primary server: %s",
308 appname, err)));
311 * Save user-visible connection string. This clobbers the original
312 * conninfo, for security. Also save host and port of the sender server
313 * this walreceiver is connected to.
315 tmp_conninfo = walrcv_get_conninfo(wrconn);
316 walrcv_get_senderinfo(wrconn, &sender_host, &sender_port);
317 SpinLockAcquire(&walrcv->mutex);
318 memset(walrcv->conninfo, 0, MAXCONNINFO);
319 if (tmp_conninfo)
320 strlcpy((char *) walrcv->conninfo, tmp_conninfo, MAXCONNINFO);
322 memset(walrcv->sender_host, 0, NI_MAXHOST);
323 if (sender_host)
324 strlcpy((char *) walrcv->sender_host, sender_host, NI_MAXHOST);
326 walrcv->sender_port = sender_port;
327 walrcv->ready_to_display = true;
328 SpinLockRelease(&walrcv->mutex);
330 if (tmp_conninfo)
331 pfree(tmp_conninfo);
333 if (sender_host)
334 pfree(sender_host);
336 first_stream = true;
337 for (;;)
339 char *primary_sysid;
340 char standby_sysid[32];
341 WalRcvStreamOptions options;
344 * Check that we're connected to a valid server using the
345 * IDENTIFY_SYSTEM replication command.
347 primary_sysid = walrcv_identify_system(wrconn, &primaryTLI);
349 snprintf(standby_sysid, sizeof(standby_sysid), UINT64_FORMAT,
350 GetSystemIdentifier());
351 if (strcmp(primary_sysid, standby_sysid) != 0)
353 ereport(ERROR,
354 (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
355 errmsg("database system identifier differs between the primary and standby"),
356 errdetail("The primary's identifier is %s, the standby's identifier is %s.",
357 primary_sysid, standby_sysid)));
361 * Confirm that the current timeline of the primary is the same or
362 * ahead of ours.
364 if (primaryTLI < startpointTLI)
365 ereport(ERROR,
366 (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
367 errmsg("highest timeline %u of the primary is behind recovery timeline %u",
368 primaryTLI, startpointTLI)));
371 * Get any missing history files. We do this always, even when we're
372 * not interested in that timeline, so that if we're promoted to
373 * become the primary later on, we don't select the same timeline that
374 * was already used in the current primary. This isn't bullet-proof -
375 * you'll need some external software to manage your cluster if you
376 * need to ensure that a unique timeline id is chosen in every case,
377 * but let's avoid the confusion of timeline id collisions where we
378 * can.
380 WalRcvFetchTimeLineHistoryFiles(startpointTLI, primaryTLI);
383 * Create temporary replication slot if requested, and update slot
384 * name in shared memory. (Note the slot name cannot already be set
385 * in this case.)
387 if (is_temp_slot)
389 snprintf(slotname, sizeof(slotname),
390 "pg_walreceiver_%lld",
391 (long long int) walrcv_get_backend_pid(wrconn));
393 walrcv_create_slot(wrconn, slotname, true, false, false, 0, NULL);
395 SpinLockAcquire(&walrcv->mutex);
396 strlcpy(walrcv->slotname, slotname, NAMEDATALEN);
397 SpinLockRelease(&walrcv->mutex);
401 * Start streaming.
403 * We'll try to start at the requested starting point and timeline,
404 * even if it's different from the server's latest timeline. In case
405 * we've already reached the end of the old timeline, the server will
406 * finish the streaming immediately, and we will go back to await
407 * orders from the startup process. If recovery_target_timeline is
408 * 'latest', the startup process will scan pg_wal and find the new
409 * history file, bump recovery target timeline, and ask us to restart
410 * on the new timeline.
412 options.logical = false;
413 options.startpoint = startpoint;
414 options.slotname = slotname[0] != '\0' ? slotname : NULL;
415 options.proto.physical.startpointTLI = startpointTLI;
416 if (walrcv_startstreaming(wrconn, &options))
418 if (first_stream)
419 ereport(LOG,
420 (errmsg("started streaming WAL from primary at %X/%X on timeline %u",
421 LSN_FORMAT_ARGS(startpoint), startpointTLI)));
422 else
423 ereport(LOG,
424 (errmsg("restarted WAL streaming at %X/%X on timeline %u",
425 LSN_FORMAT_ARGS(startpoint), startpointTLI)));
426 first_stream = false;
428 /* Initialize LogstreamResult and buffers for processing messages */
429 LogstreamResult.Write = LogstreamResult.Flush = GetXLogReplayRecPtr(NULL);
430 initStringInfo(&reply_message);
432 /* Initialize nap wakeup times. */
433 now = GetCurrentTimestamp();
434 for (int i = 0; i < NUM_WALRCV_WAKEUPS; ++i)
435 WalRcvComputeNextWakeup(i, now);
437 /* Send initial reply/feedback messages. */
438 XLogWalRcvSendReply(true, false);
439 XLogWalRcvSendHSFeedback(true);
441 /* Loop until end-of-streaming or error */
442 for (;;)
444 char *buf;
445 int len;
446 bool endofwal = false;
447 pgsocket wait_fd = PGINVALID_SOCKET;
448 int rc;
449 TimestampTz nextWakeup;
450 long nap;
453 * Exit walreceiver if we're not in recovery. This should not
454 * happen, but cross-check the status here.
456 if (!RecoveryInProgress())
457 ereport(FATAL,
458 (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
459 errmsg("cannot continue WAL streaming, recovery has already ended")));
461 /* Process any requests or signals received recently */
462 ProcessWalRcvInterrupts();
464 if (ConfigReloadPending)
466 ConfigReloadPending = false;
467 ProcessConfigFile(PGC_SIGHUP);
468 /* recompute wakeup times */
469 now = GetCurrentTimestamp();
470 for (int i = 0; i < NUM_WALRCV_WAKEUPS; ++i)
471 WalRcvComputeNextWakeup(i, now);
472 XLogWalRcvSendHSFeedback(true);
475 /* See if we can read data immediately */
476 len = walrcv_receive(wrconn, &buf, &wait_fd);
477 if (len != 0)
480 * Process the received data, and any subsequent data we
481 * can read without blocking.
483 for (;;)
485 if (len > 0)
488 * Something was received from primary, so adjust
489 * the ping and terminate wakeup times.
491 now = GetCurrentTimestamp();
492 WalRcvComputeNextWakeup(WALRCV_WAKEUP_TERMINATE,
493 now);
494 WalRcvComputeNextWakeup(WALRCV_WAKEUP_PING, now);
495 XLogWalRcvProcessMsg(buf[0], &buf[1], len - 1,
496 startpointTLI);
498 else if (len == 0)
499 break;
500 else if (len < 0)
502 ereport(LOG,
503 (errmsg("replication terminated by primary server"),
504 errdetail("End of WAL reached on timeline %u at %X/%X.",
505 startpointTLI,
506 LSN_FORMAT_ARGS(LogstreamResult.Write))));
507 endofwal = true;
508 break;
510 len = walrcv_receive(wrconn, &buf, &wait_fd);
513 /* Let the primary know that we received some data. */
514 XLogWalRcvSendReply(false, false);
517 * If we've written some records, flush them to disk and
518 * let the startup process and primary server know about
519 * them.
521 XLogWalRcvFlush(false, startpointTLI);
524 /* Check if we need to exit the streaming loop. */
525 if (endofwal)
526 break;
528 /* Find the soonest wakeup time, to limit our nap. */
529 nextWakeup = TIMESTAMP_INFINITY;
530 for (int i = 0; i < NUM_WALRCV_WAKEUPS; ++i)
531 nextWakeup = Min(wakeup[i], nextWakeup);
533 /* Calculate the nap time, clamping as necessary. */
534 now = GetCurrentTimestamp();
535 nap = TimestampDifferenceMilliseconds(now, nextWakeup);
538 * Ideally we would reuse a WaitEventSet object repeatedly
539 * here to avoid the overheads of WaitLatchOrSocket on epoll
540 * systems, but we can't be sure that libpq (or any other
541 * walreceiver implementation) has the same socket (even if
542 * the fd is the same number, it may have been closed and
543 * reopened since the last time). In future, if there is a
544 * function for removing sockets from WaitEventSet, then we
545 * could add and remove just the socket each time, potentially
546 * avoiding some system calls.
548 Assert(wait_fd != PGINVALID_SOCKET);
549 rc = WaitLatchOrSocket(MyLatch,
550 WL_EXIT_ON_PM_DEATH | WL_SOCKET_READABLE |
551 WL_TIMEOUT | WL_LATCH_SET,
552 wait_fd,
553 nap,
554 WAIT_EVENT_WAL_RECEIVER_MAIN);
555 if (rc & WL_LATCH_SET)
557 ResetLatch(MyLatch);
558 ProcessWalRcvInterrupts();
560 if (walrcv->force_reply)
563 * The recovery process has asked us to send apply
564 * feedback now. Make sure the flag is really set to
565 * false in shared memory before sending the reply, so
566 * we don't miss a new request for a reply.
568 walrcv->force_reply = false;
569 pg_memory_barrier();
570 XLogWalRcvSendReply(true, false);
573 if (rc & WL_TIMEOUT)
576 * We didn't receive anything new. If we haven't heard
577 * anything from the server for more than
578 * wal_receiver_timeout / 2, ping the server. Also, if
579 * it's been longer than wal_receiver_status_interval
580 * since the last update we sent, send a status update to
581 * the primary anyway, to report any progress in applying
582 * WAL.
584 bool requestReply = false;
587 * Check if time since last receive from primary has
588 * reached the configured limit.
590 now = GetCurrentTimestamp();
591 if (now >= wakeup[WALRCV_WAKEUP_TERMINATE])
592 ereport(ERROR,
593 (errcode(ERRCODE_CONNECTION_FAILURE),
594 errmsg("terminating walreceiver due to timeout")));
597 * If we didn't receive anything new for half of receiver
598 * replication timeout, then ping the server.
600 if (now >= wakeup[WALRCV_WAKEUP_PING])
602 requestReply = true;
603 wakeup[WALRCV_WAKEUP_PING] = TIMESTAMP_INFINITY;
606 XLogWalRcvSendReply(requestReply, requestReply);
607 XLogWalRcvSendHSFeedback(false);
612 * The backend finished streaming. Exit streaming COPY-mode from
613 * our side, too.
615 walrcv_endstreaming(wrconn, &primaryTLI);
618 * If the server had switched to a new timeline that we didn't
619 * know about when we began streaming, fetch its timeline history
620 * file now.
622 WalRcvFetchTimeLineHistoryFiles(startpointTLI, primaryTLI);
624 else
625 ereport(LOG,
626 (errmsg("primary server contains no more WAL on requested timeline %u",
627 startpointTLI)));
630 * End of WAL reached on the requested timeline. Close the last
631 * segment, and await for new orders from the startup process.
633 if (recvFile >= 0)
635 char xlogfname[MAXFNAMELEN];
637 XLogWalRcvFlush(false, startpointTLI);
638 XLogFileName(xlogfname, recvFileTLI, recvSegNo, wal_segment_size);
639 if (close(recvFile) != 0)
640 ereport(PANIC,
641 (errcode_for_file_access(),
642 errmsg("could not close WAL segment %s: %m",
643 xlogfname)));
646 * Create .done file forcibly to prevent the streamed segment from
647 * being archived later.
649 if (XLogArchiveMode != ARCHIVE_MODE_ALWAYS)
650 XLogArchiveForceDone(xlogfname);
651 else
652 XLogArchiveNotify(xlogfname);
654 recvFile = -1;
656 elog(DEBUG1, "walreceiver ended streaming and awaits new instructions");
657 WalRcvWaitForStartPosition(&startpoint, &startpointTLI);
659 /* not reached */
663 * Wait for startup process to set receiveStart and receiveStartTLI.
665 static void
666 WalRcvWaitForStartPosition(XLogRecPtr *startpoint, TimeLineID *startpointTLI)
668 WalRcvData *walrcv = WalRcv;
669 int state;
671 SpinLockAcquire(&walrcv->mutex);
672 state = walrcv->walRcvState;
673 if (state != WALRCV_STREAMING)
675 SpinLockRelease(&walrcv->mutex);
676 if (state == WALRCV_STOPPING)
677 proc_exit(0);
678 else
679 elog(FATAL, "unexpected walreceiver state");
681 walrcv->walRcvState = WALRCV_WAITING;
682 walrcv->receiveStart = InvalidXLogRecPtr;
683 walrcv->receiveStartTLI = 0;
684 SpinLockRelease(&walrcv->mutex);
686 set_ps_display("idle");
689 * nudge startup process to notice that we've stopped streaming and are
690 * now waiting for instructions.
692 WakeupRecovery();
693 for (;;)
695 ResetLatch(MyLatch);
697 ProcessWalRcvInterrupts();
699 SpinLockAcquire(&walrcv->mutex);
700 Assert(walrcv->walRcvState == WALRCV_RESTARTING ||
701 walrcv->walRcvState == WALRCV_WAITING ||
702 walrcv->walRcvState == WALRCV_STOPPING);
703 if (walrcv->walRcvState == WALRCV_RESTARTING)
706 * No need to handle changes in primary_conninfo or
707 * primary_slot_name here. Startup process will signal us to
708 * terminate in case those change.
710 *startpoint = walrcv->receiveStart;
711 *startpointTLI = walrcv->receiveStartTLI;
712 walrcv->walRcvState = WALRCV_STREAMING;
713 SpinLockRelease(&walrcv->mutex);
714 break;
716 if (walrcv->walRcvState == WALRCV_STOPPING)
719 * We should've received SIGTERM if the startup process wants us
720 * to die, but might as well check it here too.
722 SpinLockRelease(&walrcv->mutex);
723 exit(1);
725 SpinLockRelease(&walrcv->mutex);
727 (void) WaitLatch(MyLatch, WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 0,
728 WAIT_EVENT_WAL_RECEIVER_WAIT_START);
731 if (update_process_title)
733 char activitymsg[50];
735 snprintf(activitymsg, sizeof(activitymsg), "restarting at %X/%X",
736 LSN_FORMAT_ARGS(*startpoint));
737 set_ps_display(activitymsg);
742 * Fetch any missing timeline history files between 'first' and 'last'
743 * (inclusive) from the server.
745 static void
746 WalRcvFetchTimeLineHistoryFiles(TimeLineID first, TimeLineID last)
748 TimeLineID tli;
750 for (tli = first; tli <= last; tli++)
752 /* there's no history file for timeline 1 */
753 if (tli != 1 && !existsTimeLineHistory(tli))
755 char *fname;
756 char *content;
757 int len;
758 char expectedfname[MAXFNAMELEN];
760 ereport(LOG,
761 (errmsg("fetching timeline history file for timeline %u from primary server",
762 tli)));
764 walrcv_readtimelinehistoryfile(wrconn, tli, &fname, &content, &len);
767 * Check that the filename on the primary matches what we
768 * calculated ourselves. This is just a sanity check, it should
769 * always match.
771 TLHistoryFileName(expectedfname, tli);
772 if (strcmp(fname, expectedfname) != 0)
773 ereport(ERROR,
774 (errcode(ERRCODE_PROTOCOL_VIOLATION),
775 errmsg_internal("primary reported unexpected file name for timeline history file of timeline %u",
776 tli)));
779 * Write the file to pg_wal.
781 writeTimeLineHistoryFile(tli, content, len);
784 * Mark the streamed history file as ready for archiving if
785 * archive_mode is always.
787 if (XLogArchiveMode != ARCHIVE_MODE_ALWAYS)
788 XLogArchiveForceDone(fname);
789 else
790 XLogArchiveNotify(fname);
792 pfree(fname);
793 pfree(content);
799 * Mark us as STOPPED in shared memory at exit.
801 static void
802 WalRcvDie(int code, Datum arg)
804 WalRcvData *walrcv = WalRcv;
805 TimeLineID *startpointTLI_p = (TimeLineID *) DatumGetPointer(arg);
807 Assert(*startpointTLI_p != 0);
809 /* Ensure that all WAL records received are flushed to disk */
810 XLogWalRcvFlush(true, *startpointTLI_p);
812 /* Mark ourselves inactive in shared memory */
813 SpinLockAcquire(&walrcv->mutex);
814 Assert(walrcv->walRcvState == WALRCV_STREAMING ||
815 walrcv->walRcvState == WALRCV_RESTARTING ||
816 walrcv->walRcvState == WALRCV_STARTING ||
817 walrcv->walRcvState == WALRCV_WAITING ||
818 walrcv->walRcvState == WALRCV_STOPPING);
819 Assert(walrcv->pid == MyProcPid);
820 walrcv->walRcvState = WALRCV_STOPPED;
821 walrcv->pid = 0;
822 walrcv->ready_to_display = false;
823 walrcv->latch = NULL;
824 SpinLockRelease(&walrcv->mutex);
826 ConditionVariableBroadcast(&walrcv->walRcvStoppedCV);
828 /* Terminate the connection gracefully. */
829 if (wrconn != NULL)
830 walrcv_disconnect(wrconn);
832 /* Wake up the startup process to notice promptly that we're gone */
833 WakeupRecovery();
837 * Accept the message from XLOG stream, and process it.
839 static void
840 XLogWalRcvProcessMsg(unsigned char type, char *buf, Size len, TimeLineID tli)
842 int hdrlen;
843 XLogRecPtr dataStart;
844 XLogRecPtr walEnd;
845 TimestampTz sendTime;
846 bool replyRequested;
848 switch (type)
850 case 'w': /* WAL records */
852 StringInfoData incoming_message;
854 hdrlen = sizeof(int64) + sizeof(int64) + sizeof(int64);
855 if (len < hdrlen)
856 ereport(ERROR,
857 (errcode(ERRCODE_PROTOCOL_VIOLATION),
858 errmsg_internal("invalid WAL message received from primary")));
860 /* initialize a StringInfo with the given buffer */
861 initReadOnlyStringInfo(&incoming_message, buf, hdrlen);
863 /* read the fields */
864 dataStart = pq_getmsgint64(&incoming_message);
865 walEnd = pq_getmsgint64(&incoming_message);
866 sendTime = pq_getmsgint64(&incoming_message);
867 ProcessWalSndrMessage(walEnd, sendTime);
869 buf += hdrlen;
870 len -= hdrlen;
871 XLogWalRcvWrite(buf, len, dataStart, tli);
872 break;
874 case 'k': /* Keepalive */
876 StringInfoData incoming_message;
878 hdrlen = sizeof(int64) + sizeof(int64) + sizeof(char);
879 if (len != hdrlen)
880 ereport(ERROR,
881 (errcode(ERRCODE_PROTOCOL_VIOLATION),
882 errmsg_internal("invalid keepalive message received from primary")));
884 /* initialize a StringInfo with the given buffer */
885 initReadOnlyStringInfo(&incoming_message, buf, hdrlen);
887 /* read the fields */
888 walEnd = pq_getmsgint64(&incoming_message);
889 sendTime = pq_getmsgint64(&incoming_message);
890 replyRequested = pq_getmsgbyte(&incoming_message);
892 ProcessWalSndrMessage(walEnd, sendTime);
894 /* If the primary requested a reply, send one immediately */
895 if (replyRequested)
896 XLogWalRcvSendReply(true, false);
897 break;
899 default:
900 ereport(ERROR,
901 (errcode(ERRCODE_PROTOCOL_VIOLATION),
902 errmsg_internal("invalid replication message type %d",
903 type)));
908 * Write XLOG data to disk.
910 static void
911 XLogWalRcvWrite(char *buf, Size nbytes, XLogRecPtr recptr, TimeLineID tli)
913 int startoff;
914 int byteswritten;
916 Assert(tli != 0);
918 while (nbytes > 0)
920 int segbytes;
922 /* Close the current segment if it's completed */
923 if (recvFile >= 0 && !XLByteInSeg(recptr, recvSegNo, wal_segment_size))
924 XLogWalRcvClose(recptr, tli);
926 if (recvFile < 0)
928 /* Create/use new log file */
929 XLByteToSeg(recptr, recvSegNo, wal_segment_size);
930 recvFile = XLogFileInit(recvSegNo, tli);
931 recvFileTLI = tli;
934 /* Calculate the start offset of the received logs */
935 startoff = XLogSegmentOffset(recptr, wal_segment_size);
937 if (startoff + nbytes > wal_segment_size)
938 segbytes = wal_segment_size - startoff;
939 else
940 segbytes = nbytes;
942 /* OK to write the logs */
943 errno = 0;
945 byteswritten = pg_pwrite(recvFile, buf, segbytes, (off_t) startoff);
946 if (byteswritten <= 0)
948 char xlogfname[MAXFNAMELEN];
949 int save_errno;
951 /* if write didn't set errno, assume no disk space */
952 if (errno == 0)
953 errno = ENOSPC;
955 save_errno = errno;
956 XLogFileName(xlogfname, recvFileTLI, recvSegNo, wal_segment_size);
957 errno = save_errno;
958 ereport(PANIC,
959 (errcode_for_file_access(),
960 errmsg("could not write to WAL segment %s "
961 "at offset %d, length %lu: %m",
962 xlogfname, startoff, (unsigned long) segbytes)));
965 /* Update state for write */
966 recptr += byteswritten;
968 nbytes -= byteswritten;
969 buf += byteswritten;
971 LogstreamResult.Write = recptr;
974 /* Update shared-memory status */
975 pg_atomic_write_u64(&WalRcv->writtenUpto, LogstreamResult.Write);
978 * Close the current segment if it's fully written up in the last cycle of
979 * the loop, to create its archive notification file soon. Otherwise WAL
980 * archiving of the segment will be delayed until any data in the next
981 * segment is received and written.
983 if (recvFile >= 0 && !XLByteInSeg(recptr, recvSegNo, wal_segment_size))
984 XLogWalRcvClose(recptr, tli);
988 * Flush the log to disk.
990 * If we're in the midst of dying, it's unwise to do anything that might throw
991 * an error, so we skip sending a reply in that case.
993 static void
994 XLogWalRcvFlush(bool dying, TimeLineID tli)
996 Assert(tli != 0);
998 if (LogstreamResult.Flush < LogstreamResult.Write)
1000 WalRcvData *walrcv = WalRcv;
1002 issue_xlog_fsync(recvFile, recvSegNo, tli);
1004 LogstreamResult.Flush = LogstreamResult.Write;
1006 /* Update shared-memory status */
1007 SpinLockAcquire(&walrcv->mutex);
1008 if (walrcv->flushedUpto < LogstreamResult.Flush)
1010 walrcv->latestChunkStart = walrcv->flushedUpto;
1011 walrcv->flushedUpto = LogstreamResult.Flush;
1012 walrcv->receivedTLI = tli;
1014 SpinLockRelease(&walrcv->mutex);
1016 /* Signal the startup process and walsender that new WAL has arrived */
1017 WakeupRecovery();
1018 if (AllowCascadeReplication())
1019 WalSndWakeup(true, false);
1021 /* Report XLOG streaming progress in PS display */
1022 if (update_process_title)
1024 char activitymsg[50];
1026 snprintf(activitymsg, sizeof(activitymsg), "streaming %X/%X",
1027 LSN_FORMAT_ARGS(LogstreamResult.Write));
1028 set_ps_display(activitymsg);
1031 /* Also let the primary know that we made some progress */
1032 if (!dying)
1034 XLogWalRcvSendReply(false, false);
1035 XLogWalRcvSendHSFeedback(false);
1041 * Close the current segment.
1043 * Flush the segment to disk before closing it. Otherwise we have to
1044 * reopen and fsync it later.
1046 * Create an archive notification file since the segment is known completed.
1048 static void
1049 XLogWalRcvClose(XLogRecPtr recptr, TimeLineID tli)
1051 char xlogfname[MAXFNAMELEN];
1053 Assert(recvFile >= 0 && !XLByteInSeg(recptr, recvSegNo, wal_segment_size));
1054 Assert(tli != 0);
1057 * fsync() and close current file before we switch to next one. We would
1058 * otherwise have to reopen this file to fsync it later
1060 XLogWalRcvFlush(false, tli);
1062 XLogFileName(xlogfname, recvFileTLI, recvSegNo, wal_segment_size);
1065 * XLOG segment files will be re-read by recovery in startup process soon,
1066 * so we don't advise the OS to release cache pages associated with the
1067 * file like XLogFileClose() does.
1069 if (close(recvFile) != 0)
1070 ereport(PANIC,
1071 (errcode_for_file_access(),
1072 errmsg("could not close WAL segment %s: %m",
1073 xlogfname)));
1076 * Create .done file forcibly to prevent the streamed segment from being
1077 * archived later.
1079 if (XLogArchiveMode != ARCHIVE_MODE_ALWAYS)
1080 XLogArchiveForceDone(xlogfname);
1081 else
1082 XLogArchiveNotify(xlogfname);
1084 recvFile = -1;
1088 * Send reply message to primary, indicating our current WAL locations, oldest
1089 * xmin and the current time.
1091 * If 'force' is not set, the message is only sent if enough time has
1092 * passed since last status update to reach wal_receiver_status_interval.
1093 * If wal_receiver_status_interval is disabled altogether and 'force' is
1094 * false, this is a no-op.
1096 * If 'requestReply' is true, requests the server to reply immediately upon
1097 * receiving this message. This is used for heartbeats, when approaching
1098 * wal_receiver_timeout.
1100 static void
1101 XLogWalRcvSendReply(bool force, bool requestReply)
1103 static XLogRecPtr writePtr = 0;
1104 static XLogRecPtr flushPtr = 0;
1105 XLogRecPtr applyPtr;
1106 TimestampTz now;
1109 * If the user doesn't want status to be reported to the primary, be sure
1110 * to exit before doing anything at all.
1112 if (!force && wal_receiver_status_interval <= 0)
1113 return;
1115 /* Get current timestamp. */
1116 now = GetCurrentTimestamp();
1119 * We can compare the write and flush positions to the last message we
1120 * sent without taking any lock, but the apply position requires a spin
1121 * lock, so we don't check that unless something else has changed or 10
1122 * seconds have passed. This means that the apply WAL location will
1123 * appear, from the primary's point of view, to lag slightly, but since
1124 * this is only for reporting purposes and only on idle systems, that's
1125 * probably OK.
1127 if (!force
1128 && writePtr == LogstreamResult.Write
1129 && flushPtr == LogstreamResult.Flush
1130 && now < wakeup[WALRCV_WAKEUP_REPLY])
1131 return;
1133 /* Make sure we wake up when it's time to send another reply. */
1134 WalRcvComputeNextWakeup(WALRCV_WAKEUP_REPLY, now);
1136 /* Construct a new message */
1137 writePtr = LogstreamResult.Write;
1138 flushPtr = LogstreamResult.Flush;
1139 applyPtr = GetXLogReplayRecPtr(NULL);
1141 resetStringInfo(&reply_message);
1142 pq_sendbyte(&reply_message, 'r');
1143 pq_sendint64(&reply_message, writePtr);
1144 pq_sendint64(&reply_message, flushPtr);
1145 pq_sendint64(&reply_message, applyPtr);
1146 pq_sendint64(&reply_message, GetCurrentTimestamp());
1147 pq_sendbyte(&reply_message, requestReply ? 1 : 0);
1149 /* Send it */
1150 elog(DEBUG2, "sending write %X/%X flush %X/%X apply %X/%X%s",
1151 LSN_FORMAT_ARGS(writePtr),
1152 LSN_FORMAT_ARGS(flushPtr),
1153 LSN_FORMAT_ARGS(applyPtr),
1154 requestReply ? " (reply requested)" : "");
1156 walrcv_send(wrconn, reply_message.data, reply_message.len);
1160 * Send hot standby feedback message to primary, plus the current time,
1161 * in case they don't have a watch.
1163 * If the user disables feedback, send one final message to tell sender
1164 * to forget about the xmin on this standby. We also send this message
1165 * on first connect because a previous connection might have set xmin
1166 * on a replication slot. (If we're not using a slot it's harmless to
1167 * send a feedback message explicitly setting InvalidTransactionId).
1169 static void
1170 XLogWalRcvSendHSFeedback(bool immed)
1172 TimestampTz now;
1173 FullTransactionId nextFullXid;
1174 TransactionId nextXid;
1175 uint32 xmin_epoch,
1176 catalog_xmin_epoch;
1177 TransactionId xmin,
1178 catalog_xmin;
1180 /* initially true so we always send at least one feedback message */
1181 static bool primary_has_standby_xmin = true;
1184 * If the user doesn't want status to be reported to the primary, be sure
1185 * to exit before doing anything at all.
1187 if ((wal_receiver_status_interval <= 0 || !hot_standby_feedback) &&
1188 !primary_has_standby_xmin)
1189 return;
1191 /* Get current timestamp. */
1192 now = GetCurrentTimestamp();
1194 /* Send feedback at most once per wal_receiver_status_interval. */
1195 if (!immed && now < wakeup[WALRCV_WAKEUP_HSFEEDBACK])
1196 return;
1198 /* Make sure we wake up when it's time to send feedback again. */
1199 WalRcvComputeNextWakeup(WALRCV_WAKEUP_HSFEEDBACK, now);
1202 * If Hot Standby is not yet accepting connections there is nothing to
1203 * send. Check this after the interval has expired to reduce number of
1204 * calls.
1206 * Bailing out here also ensures that we don't send feedback until we've
1207 * read our own replication slot state, so we don't tell the primary to
1208 * discard needed xmin or catalog_xmin from any slots that may exist on
1209 * this replica.
1211 if (!HotStandbyActive())
1212 return;
1215 * Make the expensive call to get the oldest xmin once we are certain
1216 * everything else has been checked.
1218 if (hot_standby_feedback)
1220 GetReplicationHorizons(&xmin, &catalog_xmin);
1222 else
1224 xmin = InvalidTransactionId;
1225 catalog_xmin = InvalidTransactionId;
1229 * Get epoch and adjust if nextXid and oldestXmin are different sides of
1230 * the epoch boundary.
1232 nextFullXid = ReadNextFullTransactionId();
1233 nextXid = XidFromFullTransactionId(nextFullXid);
1234 xmin_epoch = EpochFromFullTransactionId(nextFullXid);
1235 catalog_xmin_epoch = xmin_epoch;
1236 if (nextXid < xmin)
1237 xmin_epoch--;
1238 if (nextXid < catalog_xmin)
1239 catalog_xmin_epoch--;
1241 elog(DEBUG2, "sending hot standby feedback xmin %u epoch %u catalog_xmin %u catalog_xmin_epoch %u",
1242 xmin, xmin_epoch, catalog_xmin, catalog_xmin_epoch);
1244 /* Construct the message and send it. */
1245 resetStringInfo(&reply_message);
1246 pq_sendbyte(&reply_message, 'h');
1247 pq_sendint64(&reply_message, GetCurrentTimestamp());
1248 pq_sendint32(&reply_message, xmin);
1249 pq_sendint32(&reply_message, xmin_epoch);
1250 pq_sendint32(&reply_message, catalog_xmin);
1251 pq_sendint32(&reply_message, catalog_xmin_epoch);
1252 walrcv_send(wrconn, reply_message.data, reply_message.len);
1253 if (TransactionIdIsValid(xmin) || TransactionIdIsValid(catalog_xmin))
1254 primary_has_standby_xmin = true;
1255 else
1256 primary_has_standby_xmin = false;
1260 * Update shared memory status upon receiving a message from primary.
1262 * 'walEnd' and 'sendTime' are the end-of-WAL and timestamp of the latest
1263 * message, reported by primary.
1265 static void
1266 ProcessWalSndrMessage(XLogRecPtr walEnd, TimestampTz sendTime)
1268 WalRcvData *walrcv = WalRcv;
1269 TimestampTz lastMsgReceiptTime = GetCurrentTimestamp();
1271 /* Update shared-memory status */
1272 SpinLockAcquire(&walrcv->mutex);
1273 if (walrcv->latestWalEnd < walEnd)
1274 walrcv->latestWalEndTime = sendTime;
1275 walrcv->latestWalEnd = walEnd;
1276 walrcv->lastMsgSendTime = sendTime;
1277 walrcv->lastMsgReceiptTime = lastMsgReceiptTime;
1278 SpinLockRelease(&walrcv->mutex);
1280 if (message_level_is_interesting(DEBUG2))
1282 char *sendtime;
1283 char *receipttime;
1284 int applyDelay;
1286 /* Copy because timestamptz_to_str returns a static buffer */
1287 sendtime = pstrdup(timestamptz_to_str(sendTime));
1288 receipttime = pstrdup(timestamptz_to_str(lastMsgReceiptTime));
1289 applyDelay = GetReplicationApplyDelay();
1291 /* apply delay is not available */
1292 if (applyDelay == -1)
1293 elog(DEBUG2, "sendtime %s receipttime %s replication apply delay (N/A) transfer latency %d ms",
1294 sendtime,
1295 receipttime,
1296 GetReplicationTransferLatency());
1297 else
1298 elog(DEBUG2, "sendtime %s receipttime %s replication apply delay %d ms transfer latency %d ms",
1299 sendtime,
1300 receipttime,
1301 applyDelay,
1302 GetReplicationTransferLatency());
1304 pfree(sendtime);
1305 pfree(receipttime);
1310 * Compute the next wakeup time for a given wakeup reason. Can be called to
1311 * initialize a wakeup time, to adjust it for the next wakeup, or to
1312 * reinitialize it when GUCs have changed. We ask the caller to pass in the
1313 * value of "now" because this frequently avoids multiple calls of
1314 * GetCurrentTimestamp(). It had better be a reasonably up-to-date value
1315 * though.
1317 static void
1318 WalRcvComputeNextWakeup(WalRcvWakeupReason reason, TimestampTz now)
1320 switch (reason)
1322 case WALRCV_WAKEUP_TERMINATE:
1323 if (wal_receiver_timeout <= 0)
1324 wakeup[reason] = TIMESTAMP_INFINITY;
1325 else
1326 wakeup[reason] = TimestampTzPlusMilliseconds(now, wal_receiver_timeout);
1327 break;
1328 case WALRCV_WAKEUP_PING:
1329 if (wal_receiver_timeout <= 0)
1330 wakeup[reason] = TIMESTAMP_INFINITY;
1331 else
1332 wakeup[reason] = TimestampTzPlusMilliseconds(now, wal_receiver_timeout / 2);
1333 break;
1334 case WALRCV_WAKEUP_HSFEEDBACK:
1335 if (!hot_standby_feedback || wal_receiver_status_interval <= 0)
1336 wakeup[reason] = TIMESTAMP_INFINITY;
1337 else
1338 wakeup[reason] = TimestampTzPlusSeconds(now, wal_receiver_status_interval);
1339 break;
1340 case WALRCV_WAKEUP_REPLY:
1341 if (wal_receiver_status_interval <= 0)
1342 wakeup[reason] = TIMESTAMP_INFINITY;
1343 else
1344 wakeup[reason] = TimestampTzPlusSeconds(now, wal_receiver_status_interval);
1345 break;
1346 /* there's intentionally no default: here */
1351 * Wake up the walreceiver main loop.
1353 * This is called by the startup process whenever interesting xlog records
1354 * are applied, so that walreceiver can check if it needs to send an apply
1355 * notification back to the primary which may be waiting in a COMMIT with
1356 * synchronous_commit = remote_apply.
1358 void
1359 WalRcvForceReply(void)
1361 Latch *latch;
1363 WalRcv->force_reply = true;
1364 /* fetching the latch pointer might not be atomic, so use spinlock */
1365 SpinLockAcquire(&WalRcv->mutex);
1366 latch = WalRcv->latch;
1367 SpinLockRelease(&WalRcv->mutex);
1368 if (latch)
1369 SetLatch(latch);
1373 * Return a string constant representing the state. This is used
1374 * in system functions and views, and should *not* be translated.
1376 static const char *
1377 WalRcvGetStateString(WalRcvState state)
1379 switch (state)
1381 case WALRCV_STOPPED:
1382 return "stopped";
1383 case WALRCV_STARTING:
1384 return "starting";
1385 case WALRCV_STREAMING:
1386 return "streaming";
1387 case WALRCV_WAITING:
1388 return "waiting";
1389 case WALRCV_RESTARTING:
1390 return "restarting";
1391 case WALRCV_STOPPING:
1392 return "stopping";
1394 return "UNKNOWN";
1398 * Returns activity of WAL receiver, including pid, state and xlog locations
1399 * received from the WAL sender of another server.
1401 Datum
1402 pg_stat_get_wal_receiver(PG_FUNCTION_ARGS)
1404 TupleDesc tupdesc;
1405 Datum *values;
1406 bool *nulls;
1407 int pid;
1408 bool ready_to_display;
1409 WalRcvState state;
1410 XLogRecPtr receive_start_lsn;
1411 TimeLineID receive_start_tli;
1412 XLogRecPtr written_lsn;
1413 XLogRecPtr flushed_lsn;
1414 TimeLineID received_tli;
1415 TimestampTz last_send_time;
1416 TimestampTz last_receipt_time;
1417 XLogRecPtr latest_end_lsn;
1418 TimestampTz latest_end_time;
1419 char sender_host[NI_MAXHOST];
1420 int sender_port = 0;
1421 char slotname[NAMEDATALEN];
1422 char conninfo[MAXCONNINFO];
1424 /* Take a lock to ensure value consistency */
1425 SpinLockAcquire(&WalRcv->mutex);
1426 pid = (int) WalRcv->pid;
1427 ready_to_display = WalRcv->ready_to_display;
1428 state = WalRcv->walRcvState;
1429 receive_start_lsn = WalRcv->receiveStart;
1430 receive_start_tli = WalRcv->receiveStartTLI;
1431 flushed_lsn = WalRcv->flushedUpto;
1432 received_tli = WalRcv->receivedTLI;
1433 last_send_time = WalRcv->lastMsgSendTime;
1434 last_receipt_time = WalRcv->lastMsgReceiptTime;
1435 latest_end_lsn = WalRcv->latestWalEnd;
1436 latest_end_time = WalRcv->latestWalEndTime;
1437 strlcpy(slotname, (char *) WalRcv->slotname, sizeof(slotname));
1438 strlcpy(sender_host, (char *) WalRcv->sender_host, sizeof(sender_host));
1439 sender_port = WalRcv->sender_port;
1440 strlcpy(conninfo, (char *) WalRcv->conninfo, sizeof(conninfo));
1441 SpinLockRelease(&WalRcv->mutex);
1444 * No WAL receiver (or not ready yet), just return a tuple with NULL
1445 * values
1447 if (pid == 0 || !ready_to_display)
1448 PG_RETURN_NULL();
1451 * Read "writtenUpto" without holding a spinlock. Note that it may not be
1452 * consistent with the other shared variables of the WAL receiver
1453 * protected by a spinlock, but this should not be used for data integrity
1454 * checks.
1456 written_lsn = pg_atomic_read_u64(&WalRcv->writtenUpto);
1458 /* determine result type */
1459 if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
1460 elog(ERROR, "return type must be a row type");
1462 values = palloc0(sizeof(Datum) * tupdesc->natts);
1463 nulls = palloc0(sizeof(bool) * tupdesc->natts);
1465 /* Fetch values */
1466 values[0] = Int32GetDatum(pid);
1468 if (!has_privs_of_role(GetUserId(), ROLE_PG_READ_ALL_STATS))
1471 * Only superusers and roles with privileges of pg_read_all_stats can
1472 * see details. Other users only get the pid value to know whether it
1473 * is a WAL receiver, but no details.
1475 memset(&nulls[1], true, sizeof(bool) * (tupdesc->natts - 1));
1477 else
1479 values[1] = CStringGetTextDatum(WalRcvGetStateString(state));
1481 if (XLogRecPtrIsInvalid(receive_start_lsn))
1482 nulls[2] = true;
1483 else
1484 values[2] = LSNGetDatum(receive_start_lsn);
1485 values[3] = Int32GetDatum(receive_start_tli);
1486 if (XLogRecPtrIsInvalid(written_lsn))
1487 nulls[4] = true;
1488 else
1489 values[4] = LSNGetDatum(written_lsn);
1490 if (XLogRecPtrIsInvalid(flushed_lsn))
1491 nulls[5] = true;
1492 else
1493 values[5] = LSNGetDatum(flushed_lsn);
1494 values[6] = Int32GetDatum(received_tli);
1495 if (last_send_time == 0)
1496 nulls[7] = true;
1497 else
1498 values[7] = TimestampTzGetDatum(last_send_time);
1499 if (last_receipt_time == 0)
1500 nulls[8] = true;
1501 else
1502 values[8] = TimestampTzGetDatum(last_receipt_time);
1503 if (XLogRecPtrIsInvalid(latest_end_lsn))
1504 nulls[9] = true;
1505 else
1506 values[9] = LSNGetDatum(latest_end_lsn);
1507 if (latest_end_time == 0)
1508 nulls[10] = true;
1509 else
1510 values[10] = TimestampTzGetDatum(latest_end_time);
1511 if (*slotname == '\0')
1512 nulls[11] = true;
1513 else
1514 values[11] = CStringGetTextDatum(slotname);
1515 if (*sender_host == '\0')
1516 nulls[12] = true;
1517 else
1518 values[12] = CStringGetTextDatum(sender_host);
1519 if (sender_port == 0)
1520 nulls[13] = true;
1521 else
1522 values[13] = Int32GetDatum(sender_port);
1523 if (*conninfo == '\0')
1524 nulls[14] = true;
1525 else
1526 values[14] = CStringGetTextDatum(conninfo);
1529 /* Returns the record as Datum */
1530 PG_RETURN_DATUM(HeapTupleGetDatum(heap_form_tuple(tupdesc, values, nulls)));