1
0
mirror of https://github.com/postgres/postgres.git synced 2025-07-27 12:41:57 +03:00

Display the leader apply worker's PID for parallel apply workers.

Add leader_pid to pg_stat_subscription. leader_pid is the process ID of
the leader apply worker if this process is a parallel apply worker. If
this field is NULL, it indicates that the process is a leader apply
worker or a synchronization worker. The new column makes it easier to
distinguish parallel apply workers from other kinds of workers and helps
to identify the leader for the parallel workers corresponding to a
particular subscription.

Additionally, update the leader_pid column in pg_stat_activity as well to
display the PID of the leader apply worker for parallel apply workers.

Author: Hou Zhijie
Reviewed-by: Peter Smith, Sawada Masahiko, Amit Kapila, Shveta Mallik
Discussion: https://postgr.es/m/CAA4eK1+wyN6zpaHUkCLorEWNx75MG0xhMwcFhvjqm2KURZEAGw@mail.gmail.com
This commit is contained in:
Amit Kapila
2023-01-18 09:03:12 +05:30
parent 14bdb3f13d
commit d540a02a72
11 changed files with 106 additions and 44 deletions

View File

@ -1692,7 +1692,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscription. A disabled subscription or a crashed subscription will have subscription. A disabled subscription or a crashed subscription will have
zero rows in this view. If the initial data synchronization of any zero rows in this view. If the initial data synchronization of any
table is in progress, there will be additional workers for the tables table is in progress, there will be additional workers for the tables
being synchronized. being synchronized. Moreover, if the streaming transaction is applied in
parallel, there may be additional parallel apply workers.
</para> </para>
</sect1> </sect1>

View File

@ -743,9 +743,11 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<structfield>leader_pid</structfield> <type>integer</type> <structfield>leader_pid</structfield> <type>integer</type>
</para> </para>
<para> <para>
Process ID of the parallel group leader, if this process is a Process ID of the parallel group leader if this process is a parallel
parallel query worker. <literal>NULL</literal> if this process is a query worker, or process ID of the leader apply worker if this process
parallel group leader or does not participate in parallel query. is a parallel apply worker. <literal>NULL</literal> indicates that this
process is a parallel group leader or leader apply worker, or does not
participate in any parallel operation.
</para></entry> </para></entry>
</row> </row>
@ -3206,13 +3208,24 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
</para></entry> </para></entry>
</row> </row>
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>leader_pid</structfield> <type>integer</type>
</para>
<para>
Process ID of the leader apply worker if this process is a parallel
apply worker; NULL if this process is a leader apply worker or a
synchronization worker
</para></entry>
</row>
<row> <row>
<entry role="catalog_table_entry"><para role="column_definition"> <entry role="catalog_table_entry"><para role="column_definition">
<structfield>relid</structfield> <type>oid</type> <structfield>relid</structfield> <type>oid</type>
</para> </para>
<para> <para>
OID of the relation that the worker is synchronizing; null for the OID of the relation that the worker is synchronizing; NULL for the
main apply worker leader apply worker and parallel apply workers
</para></entry> </para></entry>
</row> </row>
@ -3222,7 +3235,7 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
</para> </para>
<para> <para>
Last write-ahead log location received, the initial value of Last write-ahead log location received, the initial value of
this field being 0 this field being 0; NULL for parallel apply workers
</para></entry> </para></entry>
</row> </row>
@ -3231,7 +3244,8 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
<structfield>last_msg_send_time</structfield> <type>timestamp with time zone</type> <structfield>last_msg_send_time</structfield> <type>timestamp with time zone</type>
</para> </para>
<para> <para>
Send time of last message received from origin WAL sender Send time of last message received from origin WAL sender; NULL for
parallel apply workers
</para></entry> </para></entry>
</row> </row>
@ -3240,7 +3254,8 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
<structfield>last_msg_receipt_time</structfield> <type>timestamp with time zone</type> <structfield>last_msg_receipt_time</structfield> <type>timestamp with time zone</type>
</para> </para>
<para> <para>
Receipt time of last message received from origin WAL sender Receipt time of last message received from origin WAL sender; NULL for
parallel apply workers
</para></entry> </para></entry>
</row> </row>
@ -3249,7 +3264,8 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
<structfield>latest_end_lsn</structfield> <type>pg_lsn</type> <structfield>latest_end_lsn</structfield> <type>pg_lsn</type>
</para> </para>
<para> <para>
Last write-ahead log location reported to origin WAL sender Last write-ahead log location reported to origin WAL sender; NULL for
parallel apply workers
</para></entry> </para></entry>
</row> </row>
@ -3259,7 +3275,7 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
</para> </para>
<para> <para>
Time of last write-ahead log location reported to origin WAL Time of last write-ahead log location reported to origin WAL
sender sender; NULL for parallel apply workers
</para></entry> </para></entry>
</row> </row>
</tbody> </tbody>

View File

@ -948,6 +948,7 @@ CREATE VIEW pg_stat_subscription AS
su.oid AS subid, su.oid AS subid,
su.subname, su.subname,
st.pid, st.pid,
st.leader_pid,
st.relid, st.relid,
st.received_lsn, st.received_lsn,
st.last_msg_send_time, st.last_msg_send_time,

View File

@ -849,7 +849,7 @@ LogicalParallelApplyLoop(shm_mq_handle *mqh)
static void static void
pa_shutdown(int code, Datum arg) pa_shutdown(int code, Datum arg)
{ {
SendProcSignal(MyLogicalRepWorker->apply_leader_pid, SendProcSignal(MyLogicalRepWorker->leader_pid,
PROCSIG_PARALLEL_APPLY_MESSAGE, PROCSIG_PARALLEL_APPLY_MESSAGE,
InvalidBackendId); InvalidBackendId);
@ -932,7 +932,7 @@ ParallelApplyWorkerMain(Datum main_arg)
error_mqh = shm_mq_attach(mq, seg, NULL); error_mqh = shm_mq_attach(mq, seg, NULL);
pq_redirect_to_shm_mq(seg, error_mqh); pq_redirect_to_shm_mq(seg, error_mqh);
pq_set_parallel_leader(MyLogicalRepWorker->apply_leader_pid, pq_set_parallel_leader(MyLogicalRepWorker->leader_pid,
InvalidBackendId); InvalidBackendId);
MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time = MyLogicalRepWorker->last_send_time = MyLogicalRepWorker->last_recv_time =
@ -950,7 +950,7 @@ ParallelApplyWorkerMain(Datum main_arg)
* The parallel apply worker doesn't need to monopolize this replication * The parallel apply worker doesn't need to monopolize this replication
* origin which was already acquired by its leader process. * origin which was already acquired by its leader process.
*/ */
replorigin_session_setup(originid, MyLogicalRepWorker->apply_leader_pid); replorigin_session_setup(originid, MyLogicalRepWorker->leader_pid);
replorigin_session_origin = originid; replorigin_session_origin = originid;
CommitTransactionCommand(); CommitTransactionCommand();

View File

@ -410,7 +410,7 @@ retry:
worker->relstate = SUBREL_STATE_UNKNOWN; worker->relstate = SUBREL_STATE_UNKNOWN;
worker->relstate_lsn = InvalidXLogRecPtr; worker->relstate_lsn = InvalidXLogRecPtr;
worker->stream_fileset = NULL; worker->stream_fileset = NULL;
worker->apply_leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid; worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker; worker->parallel_apply = is_parallel_apply_worker;
worker->last_lsn = InvalidXLogRecPtr; worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time); TIMESTAMP_NOBEGIN(worker->last_send_time);
@ -732,7 +732,7 @@ logicalrep_worker_cleanup(LogicalRepWorker *worker)
worker->userid = InvalidOid; worker->userid = InvalidOid;
worker->subid = InvalidOid; worker->subid = InvalidOid;
worker->relid = InvalidOid; worker->relid = InvalidOid;
worker->apply_leader_pid = InvalidPid; worker->leader_pid = InvalidPid;
worker->parallel_apply = false; worker->parallel_apply = false;
} }
@ -1066,13 +1066,41 @@ IsLogicalLauncher(void)
return LogicalRepCtx->launcher_pid == MyProcPid; return LogicalRepCtx->launcher_pid == MyProcPid;
} }
/*
* Return the pid of the leader apply worker if the given pid is the pid of a
* parallel apply worker, otherwise, return InvalidPid.
*/
pid_t
GetLeaderApplyWorkerPid(pid_t pid)
{
int leader_pid = InvalidPid;
int i;
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
for (i = 0; i < max_logical_replication_workers; i++)
{
LogicalRepWorker *w = &LogicalRepCtx->workers[i];
if (isParallelApplyWorker(w) && w->proc && pid == w->proc->pid)
{
leader_pid = w->leader_pid;
break;
}
}
LWLockRelease(LogicalRepWorkerLock);
return leader_pid;
}
/* /*
* Returns state of the subscriptions. * Returns state of the subscriptions.
*/ */
Datum Datum
pg_stat_get_subscription(PG_FUNCTION_ARGS) pg_stat_get_subscription(PG_FUNCTION_ARGS)
{ {
#define PG_STAT_GET_SUBSCRIPTION_COLS 8 #define PG_STAT_GET_SUBSCRIPTION_COLS 9
Oid subid = PG_ARGISNULL(0) ? InvalidOid : PG_GETARG_OID(0); Oid subid = PG_ARGISNULL(0) ? InvalidOid : PG_GETARG_OID(0);
int i; int i;
ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo; ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
@ -1098,10 +1126,6 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
if (OidIsValid(subid) && worker.subid != subid) if (OidIsValid(subid) && worker.subid != subid)
continue; continue;
/* Skip if this is a parallel apply worker */
if (isParallelApplyWorker(&worker))
continue;
worker_pid = worker.proc->pid; worker_pid = worker.proc->pid;
values[0] = ObjectIdGetDatum(worker.subid); values[0] = ObjectIdGetDatum(worker.subid);
@ -1110,26 +1134,32 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
else else
nulls[1] = true; nulls[1] = true;
values[2] = Int32GetDatum(worker_pid); values[2] = Int32GetDatum(worker_pid);
if (XLogRecPtrIsInvalid(worker.last_lsn))
nulls[3] = true; if (isParallelApplyWorker(&worker))
values[3] = Int32GetDatum(worker.leader_pid);
else else
values[3] = LSNGetDatum(worker.last_lsn); nulls[3] = true;
if (worker.last_send_time == 0)
if (XLogRecPtrIsInvalid(worker.last_lsn))
nulls[4] = true; nulls[4] = true;
else else
values[4] = TimestampTzGetDatum(worker.last_send_time); values[4] = LSNGetDatum(worker.last_lsn);
if (worker.last_recv_time == 0) if (worker.last_send_time == 0)
nulls[5] = true; nulls[5] = true;
else else
values[5] = TimestampTzGetDatum(worker.last_recv_time); values[5] = TimestampTzGetDatum(worker.last_send_time);
if (XLogRecPtrIsInvalid(worker.reply_lsn)) if (worker.last_recv_time == 0)
nulls[6] = true; nulls[6] = true;
else else
values[6] = LSNGetDatum(worker.reply_lsn); values[6] = TimestampTzGetDatum(worker.last_recv_time);
if (worker.reply_time == 0) if (XLogRecPtrIsInvalid(worker.reply_lsn))
nulls[7] = true; nulls[7] = true;
else else
values[7] = TimestampTzGetDatum(worker.reply_time); values[7] = LSNGetDatum(worker.reply_lsn);
if (worker.reply_time == 0)
nulls[8] = true;
else
values[8] = TimestampTzGetDatum(worker.reply_time);
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls); values, nulls);

View File

@ -25,6 +25,7 @@
#include "pgstat.h" #include "pgstat.h"
#include "postmaster/bgworker_internals.h" #include "postmaster/bgworker_internals.h"
#include "postmaster/postmaster.h" #include "postmaster/postmaster.h"
#include "replication/logicallauncher.h"
#include "storage/proc.h" #include "storage/proc.h"
#include "storage/procarray.h" #include "storage/procarray.h"
#include "utils/acl.h" #include "utils/acl.h"
@ -409,9 +410,9 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
/* /*
* If a PGPROC entry was retrieved, display wait events and lock * If a PGPROC entry was retrieved, display wait events and lock
* group leader information if any. To avoid extra overhead, no * group leader or apply leader information if any. To avoid
* extra lock is being held, so there is no guarantee of * extra overhead, no extra lock is being held, so there is no
* consistency across multiple rows. * guarantee of consistency across multiple rows.
*/ */
if (proc != NULL) if (proc != NULL)
{ {
@ -426,14 +427,24 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
/* /*
* Show the leader only for active parallel workers. This * Show the leader only for active parallel workers. This
* leaves the field as NULL for the leader of a parallel * leaves the field as NULL for the leader of a parallel group
* group. * or the leader of parallel apply workers.
*/ */
if (leader && leader->pid != beentry->st_procpid) if (leader && leader->pid != beentry->st_procpid)
{ {
values[28] = Int32GetDatum(leader->pid); values[28] = Int32GetDatum(leader->pid);
nulls[28] = false; nulls[28] = false;
} }
else if (beentry->st_backendType == B_BG_WORKER)
{
int leader_pid = GetLeaderApplyWorkerPid(beentry->st_procpid);
if (leader_pid != InvalidPid)
{
values[28] = Int32GetDatum(leader_pid);
nulls[28] = false;
}
}
} }
if (wait_event_type) if (wait_event_type)

View File

@ -57,6 +57,6 @@
*/ */
/* yyyymmddN */ /* yyyymmddN */
#define CATALOG_VERSION_NO 202301131 #define CATALOG_VERSION_NO 202301181
#endif #endif

View File

@ -5430,9 +5430,9 @@
proname => 'pg_stat_get_subscription', prorows => '10', proisstrict => 'f', proname => 'pg_stat_get_subscription', prorows => '10', proisstrict => 'f',
proretset => 't', provolatile => 's', proparallel => 'r', proretset => 't', provolatile => 's', proparallel => 'r',
prorettype => 'record', proargtypes => 'oid', prorettype => 'record', proargtypes => 'oid',
proallargtypes => '{oid,oid,oid,int4,pg_lsn,timestamptz,timestamptz,pg_lsn,timestamptz}', proallargtypes => '{oid,oid,oid,int4,int4,pg_lsn,timestamptz,timestamptz,pg_lsn,timestamptz}',
proargmodes => '{i,o,o,o,o,o,o,o,o}', proargmodes => '{i,o,o,o,o,o,o,o,o,o}',
proargnames => '{subid,subid,relid,pid,received_lsn,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time}', proargnames => '{subid,subid,relid,pid,leader_pid,received_lsn,last_msg_send_time,last_msg_receipt_time,latest_end_lsn,latest_end_time}',
prosrc => 'pg_stat_get_subscription' }, prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID', { oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r', proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',

View File

@ -27,4 +27,6 @@ extern void AtEOXact_ApplyLauncher(bool isCommit);
extern bool IsLogicalLauncher(void); extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
#endif /* LOGICALLAUNCHER_H */ #endif /* LOGICALLAUNCHER_H */

View File

@ -71,7 +71,7 @@ typedef struct LogicalRepWorker
* PID of leader apply worker if this slot is used for a parallel apply * PID of leader apply worker if this slot is used for a parallel apply
* worker, InvalidPid otherwise. * worker, InvalidPid otherwise.
*/ */
pid_t apply_leader_pid; pid_t leader_pid;
/* Indicates whether apply can be performed in parallel. */ /* Indicates whether apply can be performed in parallel. */
bool parallel_apply; bool parallel_apply;
@ -303,7 +303,7 @@ extern void pa_decr_and_wait_stream_block(void);
extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo, extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
XLogRecPtr remote_lsn); XLogRecPtr remote_lsn);
#define isParallelApplyWorker(worker) ((worker)->apply_leader_pid != InvalidPid) #define isParallelApplyWorker(worker) ((worker)->leader_pid != InvalidPid)
static inline bool static inline bool
am_tablesync_worker(void) am_tablesync_worker(void)

View File

@ -2094,6 +2094,7 @@ pg_stat_ssl| SELECT s.pid,
pg_stat_subscription| SELECT su.oid AS subid, pg_stat_subscription| SELECT su.oid AS subid,
su.subname, su.subname,
st.pid, st.pid,
st.leader_pid,
st.relid, st.relid,
st.received_lsn, st.received_lsn,
st.last_msg_send_time, st.last_msg_send_time,
@ -2101,7 +2102,7 @@ pg_stat_subscription| SELECT su.oid AS subid,
st.latest_end_lsn, st.latest_end_lsn,
st.latest_end_time st.latest_end_time
FROM (pg_subscription su FROM (pg_subscription su
LEFT JOIN pg_stat_get_subscription(NULL::oid) st(subid, relid, pid, received_lsn, last_msg_send_time, last_msg_receipt_time, latest_end_lsn, latest_end_time) ON ((st.subid = su.oid))); LEFT JOIN pg_stat_get_subscription(NULL::oid) st(subid, relid, pid, leader_pid, received_lsn, last_msg_send_time, last_msg_receipt_time, latest_end_lsn, latest_end_time) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid, pg_stat_subscription_stats| SELECT ss.subid,
s.subname, s.subname,
ss.apply_error_count, ss.apply_error_count,