]> granicus.if.org Git - postgresql/commitdiff
Remove the separate 'stats buffer' process, letting backend stats messages
authorTom Lane <tgl@sss.pgh.pa.us>
Thu, 29 Jun 2006 20:00:08 +0000 (20:00 +0000)
committerTom Lane <tgl@sss.pgh.pa.us>
Thu, 29 Jun 2006 20:00:08 +0000 (20:00 +0000)
be delivered directly to the collector process.  The extra process context
swaps required to transfer data through the buffer process seem to outweigh
any value the buffering might have.  Per recent discussion and tests.
I modified Bruce's draft patch to use poll() rather than select() where
available (this makes a noticeable difference on my system), and fixed
up the EXEC_BACKEND case.

doc/src/sgml/monitoring.sgml
src/backend/postmaster/pgstat.c
src/backend/postmaster/postmaster.c
src/include/pgstat.h

index 11cbb5ff33988c65ffc13854c1d54b0e7493d830..21275f9d5f0ac22dbad301dbab8134e9925f46b6 100644 (file)
@@ -1,4 +1,4 @@
-<!-- $PostgreSQL: pgsql/doc/src/sgml/monitoring.sgml,v 1.35 2006/06/27 19:07:50 momjian Exp $ -->
+<!-- $PostgreSQL: pgsql/doc/src/sgml/monitoring.sgml,v 1.36 2006/06/29 20:00:08 tgl Exp $ -->
 
 <chapter id="monitoring">
  <title>Monitoring Database Activity</title>
@@ -50,7 +50,7 @@
 <screen>
 $ ps auxww | grep ^postgres
 postgres   960  0.0  1.1  6104 1480 pts/1    SN   13:17   0:00 postgres -i
-postgres   963  0.0  1.1  7084 1472 pts/1    SN   13:17   0:00 postgres: stats buffer process   
+postgres   963  0.0  1.1  7084 1472 pts/1    SN   13:17   0:00 postgres: writer process
 postgres   965  0.0  1.1  6152 1512 pts/1    SN   13:17   0:00 postgres: stats collector process   
 postgres   998  0.0  2.3  6532 2992 pts/1    SN   13:18   0:00 postgres: tgl runbug 127.0.0.1 idle
 postgres  1003  0.0  2.4  6532 3128 pts/1    SN   13:19   0:00 postgres: tgl regression [local] SELECT waiting
@@ -60,10 +60,11 @@ postgres  1016  0.1  2.4  6532 3080 pts/1    SN   13:19   0:00 postgres: tgl reg
    (The appropriate invocation of <command>ps</> varies across different
    platforms, as do the details of what is shown.  This example is from a
    recent Linux system.)  The first process listed here is the
-   the master server process.  The command arguments
+   master server process.  The command arguments
    shown for it are the same ones given when it was launched.  The next two
-   processes implement the statistics collector, which will be described in
-   detail in the next section.  (These will not be present if you have set
+   processes are background worker processes automatically launched by the
+   master process.  (The <quote>stats collector</> process will not be present
+   if you have set
    the system not to start the statistics collector.)  Each of the remaining
    processes is a server process handling one client connection.  Each such
    process sets its command line display in the form
@@ -83,6 +84,13 @@ postgres: <replaceable>user</> <replaceable>database</> <replaceable>host</> <re
   thereby release some lock or other.
   </para>
 
+  <para>
+   If you have turned off <xref linkend="guc-update-process-title"> then the
+   activity indicator is not updated; the process title is set only once
+   when a new process is launched.  On some platforms this saves a useful
+   amount of per-command overhead, on others it's insignificant.
+  </para>
+
   <tip>
   <para>
   <productname>Solaris</productname> requires special handling. You must
index 34ae644351dfeb00bc38fc90b3b076b0ab5a7248..cac931dc88e9e4b1449248318b76b723afd3f4fa 100644 (file)
@@ -13,7 +13,7 @@
  *
  *     Copyright (c) 2001-2006, PostgreSQL Global Development Group
  *
- *     $PostgreSQL: pgsql/src/backend/postmaster/pgstat.c,v 1.132 2006/06/27 22:16:43 momjian Exp $
+ *     $PostgreSQL: pgsql/src/backend/postmaster/pgstat.c,v 1.133 2006/06/29 20:00:08 tgl Exp $
  * ----------
  */
 #include "postgres.h"
 #include <arpa/inet.h>
 #include <signal.h>
 #include <time.h>
+#ifdef HAVE_POLL_H
+#include <poll.h>
+#endif
+#ifdef HAVE_SYS_POLL_H
+#include <sys/poll.h>
+#endif
 
 #include "pgstat.h"
 
                                                                                 * failed statistics collector; in
                                                                                 * seconds. */
 
-/* ----------
- * Amount of space reserved in pgstat_recvbuffer().
- * ----------
- */
-#define PGSTAT_RECVBUFFERSZ            ((int) (1024 * sizeof(PgStat_Msg)))
+#define PGSTAT_SELECT_TIMEOUT  2               /* How often to check for postmaster
+                                                                                * death; in seconds. */
+
 
 /* ----------
  * The initial size hints for the hash tables used in the collector.
@@ -102,12 +106,9 @@ bool               pgstat_collect_querystring = false;
  * ----------
  */
 NON_EXEC_STATIC int pgStatSock = -1;
-NON_EXEC_STATIC int pgStatPipe[2] = {-1, -1};
 
 static struct sockaddr_storage pgStatAddr;
 
-static pid_t pgStatCollectorPid = 0;
-
 static time_t last_pgstat_start_time;
 
 static bool pgStatRunningInCollector = false;
@@ -138,7 +139,8 @@ static TransactionId pgStatLocalStatusXact = InvalidTransactionId;
 static PgBackendStatus *localBackendStatusTable = NULL;
 static int     localNumBackends = 0;
 
-static volatile bool   need_statwrite;
+static volatile bool   need_exit = false;
+static volatile bool   need_statwrite = false;
 
 
 /* ----------
@@ -146,23 +148,12 @@ static volatile bool      need_statwrite;
  * ----------
  */
 #ifdef EXEC_BACKEND
-
-typedef enum STATS_PROCESS_TYPE
-{
-       STAT_PROC_BUFFER,
-       STAT_PROC_COLLECTOR
-}      STATS_PROCESS_TYPE;
-
-static pid_t pgstat_forkexec(STATS_PROCESS_TYPE procType);
-static void pgstat_parseArgs(int argc, char *argv[]);
+static pid_t pgstat_forkexec(void);
 #endif
 
-NON_EXEC_STATIC void PgstatBufferMain(int argc, char *argv[]);
 NON_EXEC_STATIC void PgstatCollectorMain(int argc, char *argv[]);
-static void force_statwrite(SIGNAL_ARGS);
-static void pgstat_recvbuffer(void);
 static void pgstat_exit(SIGNAL_ARGS);
-static void pgstat_die(SIGNAL_ARGS);
+static void force_statwrite(SIGNAL_ARGS);
 static void pgstat_beshutdown_hook(int code, Datum arg);
 
 static PgStat_StatDBEntry *pgstat_get_db_entry(Oid databaseid, bool create);
@@ -417,9 +408,8 @@ pgstat_init(void)
 
        /*
         * Set the socket to non-blocking IO.  This ensures that if the collector
-        * falls behind (despite the buffering process), statistics messages will
-        * be discarded; backends won't block waiting to send messages to the
-        * collector.
+        * falls behind, statistics messages will be discarded; backends won't
+        * block waiting to send messages to the collector.
         */
        if (!pg_set_noblock(pgStatSock))
        {
@@ -468,65 +458,24 @@ pgstat_reset_all(void)
 /*
  * pgstat_forkexec() -
  *
- * Format up the arglist for, then fork and exec, statistics
- * (buffer and collector) processes
+ * Format up the arglist for, then fork and exec, statistics collector process
  */
 static pid_t
-pgstat_forkexec(STATS_PROCESS_TYPE procType)
+pgstat_forkexec(void)
 {
        char       *av[10];
-       int                     ac = 0,
-                               bufc = 0,
-                               i;
-       char            pgstatBuf[2][32];
+       int                     ac = 0;
 
        av[ac++] = "postgres";
-
-       switch (procType)
-       {
-               case STAT_PROC_BUFFER:
-                       av[ac++] = "--forkbuf";
-                       break;
-
-               case STAT_PROC_COLLECTOR:
-                       av[ac++] = "--forkcol";
-                       break;
-
-               default:
-                       Assert(false);
-       }
-
+       av[ac++] = "--forkcol";
        av[ac++] = NULL;                        /* filled in by postmaster_forkexec */
 
-       /* postgres_exec_path is not passed by write_backend_variables */
-       av[ac++] = postgres_exec_path;
-
-       /* Add to the arg list */
-       Assert(bufc <= lengthof(pgstatBuf));
-       for (i = 0; i < bufc; i++)
-               av[ac++] = pgstatBuf[i];
-
        av[ac] = NULL;
        Assert(ac < lengthof(av));
 
        return postmaster_forkexec(ac, av);
 }
 
-
-/*
- * pgstat_parseArgs() -
- *
- * Extract data from the arglist for exec'ed statistics
- * (buffer and collector) processes
- */
-static void
-pgstat_parseArgs(int argc, char *argv[])
-{
-       Assert(argc == 4);
-
-       argc = 3;
-       StrNCpy(postgres_exec_path, argv[argc++], MAXPGPATH);
-}
 #endif   /* EXEC_BACKEND */
 
 
@@ -585,14 +534,14 @@ pgstat_start(void)
         * Okay, fork off the collector.
         */
 #ifdef EXEC_BACKEND
-       switch ((pgStatPid = pgstat_forkexec(STAT_PROC_BUFFER)))
+       switch ((pgStatPid = pgstat_forkexec()))
 #else
        switch ((pgStatPid = fork_process()))
 #endif
        {
                case -1:
                        ereport(LOG,
-                                       (errmsg("could not fork statistics buffer: %m")));
+                                       (errmsg("could not fork statistics collector: %m")));
                        return 0;
 
 #ifndef EXEC_BACKEND
@@ -607,7 +556,7 @@ pgstat_start(void)
                        /* Drop our connection to postmaster's shared memory, as well */
                        PGSharedMemoryDetach();
 
-                       PgstatBufferMain(0, NULL);
+                       PgstatCollectorMain(0, NULL);
                        break;
 #endif
 
@@ -1602,94 +1551,11 @@ pgstat_send(void *msg, int len)
 }
 
 
-/* ----------
- * PgstatBufferMain() -
- *
- *     Start up the statistics buffer process.  This is the body of the
- *     postmaster child process.
- *
- *     The argc/argv parameters are valid only in EXEC_BACKEND case.
- * ----------
- */
-NON_EXEC_STATIC void
-PgstatBufferMain(int argc, char *argv[])
-{
-       IsUnderPostmaster = true;       /* we are a postmaster subprocess now */
-
-       MyProcPid = getpid();           /* reset MyProcPid */
-
-       /*
-        * Ignore all signals usually bound to some action in the postmaster,
-        * except for SIGCHLD and SIGQUIT --- see pgstat_recvbuffer.
-        */
-       pqsignal(SIGHUP, SIG_IGN);
-       pqsignal(SIGINT, SIG_IGN);
-       pqsignal(SIGTERM, SIG_IGN);
-       pqsignal(SIGQUIT, pgstat_exit);
-       pqsignal(SIGALRM, SIG_IGN);
-       pqsignal(SIGPIPE, SIG_IGN);
-       pqsignal(SIGUSR1, SIG_IGN);
-       pqsignal(SIGUSR2, SIG_IGN);
-       pqsignal(SIGCHLD, pgstat_die);
-       pqsignal(SIGTTIN, SIG_DFL);
-       pqsignal(SIGTTOU, SIG_DFL);
-       pqsignal(SIGCONT, SIG_DFL);
-       pqsignal(SIGWINCH, SIG_DFL);
-       /* unblock will happen in pgstat_recvbuffer */
-
-#ifdef EXEC_BACKEND
-       pgstat_parseArgs(argc, argv);
-#endif
-
-       /*
-        * Start a buffering process to read from the socket, so we have a little
-        * more time to process incoming messages.
-        *
-        * NOTE: the process structure is: postmaster is parent of buffer process
-        * is parent of collector process.      This way, the buffer can detect
-        * collector failure via SIGCHLD, whereas otherwise it wouldn't notice
-        * collector failure until it tried to write on the pipe.  That would mean
-        * that after the postmaster started a new collector, we'd have two buffer
-        * processes competing to read from the UDP socket --- not good.
-        */
-       if (pgpipe(pgStatPipe) < 0)
-               ereport(ERROR,
-                               (errcode_for_socket_access(),
-                                errmsg("could not create pipe for statistics buffer: %m")));
-
-       /* child becomes collector process */
-#ifdef EXEC_BACKEND
-       pgStatCollectorPid = pgstat_forkexec(STAT_PROC_COLLECTOR);
-#else
-       pgStatCollectorPid = fork();
-#endif
-       switch (pgStatCollectorPid)
-       {
-               case -1:
-                       ereport(ERROR,
-                                       (errmsg("could not fork statistics collector: %m")));
-
-#ifndef EXEC_BACKEND
-               case 0:
-                       /* child becomes collector process */
-                       PgstatCollectorMain(0, NULL);
-                       break;
-#endif
-
-               default:
-                       /* parent becomes buffer process */
-                       closesocket(pgStatPipe[0]);
-                       pgstat_recvbuffer();
-       }
-       exit(0);
-}
-
-
 /* ----------
  * PgstatCollectorMain() -
  *
- *     Start up the statistics collector itself.  This is the body of the
- *     postmaster grandchild process.
+ *     Start up the statistics collector process.  This is the body of the
+ *     postmaster child process.
  *
  *     The argc/argv parameters are valid only in EXEC_BACKEND case.
  * ----------
@@ -1697,30 +1563,29 @@ PgstatBufferMain(int argc, char *argv[])
 NON_EXEC_STATIC void
 PgstatCollectorMain(int argc, char *argv[])
 {
+       struct itimerval write_timeout;
+       bool            need_timer = false;
+       int                     len;
        PgStat_Msg      msg;
+#ifdef HAVE_POLL
+       struct pollfd input_fd;
+#else
+       struct timeval sel_timeout;
        fd_set          rfds;
-       int                     readPipe;
-       int                     len = 0;
-       struct itimerval timeout;
-       bool            need_timer = false;
+#endif
+
+       IsUnderPostmaster = true;       /* we are a postmaster subprocess now */
 
        MyProcPid = getpid();           /* reset MyProcPid */
 
        /*
-        * Reset signal handling.  With the exception of restoring default SIGCHLD
-        * and SIGQUIT handling, this is a no-op in the non-EXEC_BACKEND case
-        * because we'll have inherited these settings from the buffer process;
-        * but it's not a no-op for EXEC_BACKEND.
+        * Ignore all signals usually bound to some action in the postmaster,
+        * except SIGQUIT and SIGALRM.
         */
        pqsignal(SIGHUP, SIG_IGN);
        pqsignal(SIGINT, SIG_IGN);
        pqsignal(SIGTERM, SIG_IGN);
-#ifndef WIN32
-       pqsignal(SIGQUIT, SIG_IGN);
-#else
-       /* kluge to allow buffer process to kill collector; FIXME */
        pqsignal(SIGQUIT, pgstat_exit);
-#endif
        pqsignal(SIGALRM, force_statwrite);
        pqsignal(SIGPIPE, SIG_IGN);
        pqsignal(SIGUSR1, SIG_IGN);
@@ -1732,14 +1597,6 @@ PgstatCollectorMain(int argc, char *argv[])
        pqsignal(SIGWINCH, SIG_DFL);
        PG_SETMASK(&UnBlockSig);
 
-#ifdef EXEC_BACKEND
-       pgstat_parseArgs(argc, argv);
-#endif
-
-       /* Close unwanted files */
-       closesocket(pgStatPipe[1]);
-       closesocket(pgStatSock);
-
        /*
         * Identify myself via ps
         */
@@ -1751,9 +1608,9 @@ PgstatCollectorMain(int argc, char *argv[])
        need_statwrite = true;
 
        /* Preset the delay between status file writes */
-       MemSet(&timeout, 0, sizeof(struct itimerval));
-       timeout.it_value.tv_sec = PGSTAT_STAT_INTERVAL / 1000;
-       timeout.it_value.tv_usec = PGSTAT_STAT_INTERVAL % 1000;
+       MemSet(&write_timeout, 0, sizeof(struct itimerval));
+       write_timeout.it_value.tv_sec = PGSTAT_STAT_INTERVAL / 1000;
+       write_timeout.it_value.tv_usec = PGSTAT_STAT_INTERVAL % 1000;
 
        /*
         * Read in an existing statistics stats file or initialize the stats to
@@ -1762,14 +1619,32 @@ PgstatCollectorMain(int argc, char *argv[])
        pgStatRunningInCollector = true;
        pgstat_read_statsfile(&pgStatDBHash, InvalidOid);
 
-       readPipe = pgStatPipe[0];
+       /*
+        * Setup the descriptor set for select(2).  Since only one bit in the
+        * set ever changes, we need not repeat FD_ZERO each time.
+        */
+#ifndef HAVE_POLL
+       FD_ZERO(&rfds);
+#endif
 
        /*
-        * Process incoming messages and handle all the reporting stuff until
-        * there are no more messages.
+        * Loop to process messages until we get SIGQUIT or detect ungraceful
+        * death of our parent postmaster.
+        *
+        * For performance reasons, we don't want to do a PostmasterIsAlive()
+        * test after every message; instead, do it at statwrite time and if
+        * select()/poll() is interrupted by timeout.
         */
        for (;;)
        {
+               int             got_data;
+
+               /*
+                * Quit if we get SIGQUIT from the postmaster.
+                */
+               if (need_exit)
+                       break;
+
                /*
                 * If time to write the stats file, do so.  Note that the alarm
                 * interrupt isn't re-enabled immediately, but only after we next
@@ -1778,21 +1653,53 @@ PgstatCollectorMain(int argc, char *argv[])
                 */
                if (need_statwrite)
                {
+                       /* Check for postmaster death; if so we'll write file below */
+                       if (!PostmasterIsAlive(true))
+                               break;
+
                        pgstat_write_statsfile();
                        need_statwrite = false;
                        need_timer = true;
                }
 
                /*
-                * Setup the descriptor set for select(2)
+                * Wait for a message to arrive; but not for more than
+                * PGSTAT_SELECT_TIMEOUT seconds. (This determines how quickly we will
+                * shut down after an ungraceful postmaster termination; so it needn't
+                * be very fast.  However, on some systems SIGQUIT won't interrupt
+                * the poll/select call, so this also limits speed of response to
+                * SIGQUIT, which is more important.)
+                *
+                * We use poll(2) if available, otherwise select(2)
                 */
-               FD_ZERO(&rfds);
-               FD_SET(readPipe, &rfds);
+#ifdef HAVE_POLL
+               input_fd.fd = pgStatSock;
+               input_fd.events = POLLIN | POLLERR;
+               input_fd.revents = 0;
+
+               if (poll(&input_fd, 1, PGSTAT_SELECT_TIMEOUT * 1000) < 0)
+               {
+                       if (errno == EINTR)
+                               continue;
+                       ereport(ERROR,
+                                       (errcode_for_socket_access(),
+                                        errmsg("poll() failed in statistics collector: %m")));
+               }
+
+               got_data = (input_fd.revents != 0);
+
+#else                                                  /* !HAVE_POLL */
+
+               FD_SET(pgStatSock, &rfds);
 
                /*
-                * Now wait for something to do.
+                * timeout struct is modified by select() on some operating systems,
+                * so re-fill it each time.
                 */
-               if (select(readPipe + 1, &rfds, NULL, NULL, NULL) < 0)
+               sel_timeout.tv_sec = PGSTAT_SELECT_TIMEOUT;
+               sel_timeout.tv_usec = 0;
+
+               if (select(pgStatSock + 1, &rfds, NULL, NULL, &sel_timeout) < 0)
                {
                        if (errno == EINTR)
                                continue;
@@ -1801,67 +1708,37 @@ PgstatCollectorMain(int argc, char *argv[])
                                         errmsg("select() failed in statistics collector: %m")));
                }
 
+               got_data = FD_ISSET(pgStatSock, &rfds);
+
+#endif   /* HAVE_POLL */
+
                /*
-                * Check if there is a new statistics message to collect.
+                * If there is a message on the socket, read it and check for
+                * validity.
                 */
-               if (FD_ISSET(readPipe, &rfds))
+               if (got_data)
                {
+                       len = recv(pgStatSock, (char *) &msg,
+                                          sizeof(PgStat_Msg), 0);
+                       if (len < 0)
+                               ereport(ERROR,
+                                               (errcode_for_socket_access(),
+                                                errmsg("could not read statistics message: %m")));
+
                        /*
-                        * We may need to issue multiple read calls in case the buffer
-                        * process didn't write the message in a single write, which is
-                        * possible since it dumps its buffer bytewise. In any case, we'd
-                        * need two reads since we don't know the message length
-                        * initially.
+                        * We ignore messages that are smaller than our common header
                         */
-                       int                     nread = 0;
-                       int                     targetlen = sizeof(PgStat_MsgHdr);              /* initial */
-                       bool            pipeEOF = false;
-
-                       while (nread < targetlen)
-                       {
-                               len = piperead(readPipe, ((char *) &msg) + nread,
-                                                          targetlen - nread);
-                               if (len < 0)
-                               {
-                                       if (errno == EINTR)
-                                               continue;
-                                       ereport(ERROR,
-                                                       (errcode_for_socket_access(),
-                                                        errmsg("could not read from statistics collector pipe: %m")));
-                               }
-                               if (len == 0)   /* EOF on the pipe! */
-                               {
-                                       pipeEOF = true;
-                                       break;
-                               }
-                               nread += len;
-                               if (nread == sizeof(PgStat_MsgHdr))
-                               {
-                                       /* we have the header, compute actual msg length */
-                                       targetlen = msg.msg_hdr.m_size;
-                                       if (targetlen < (int) sizeof(PgStat_MsgHdr) ||
-                                               targetlen > (int) sizeof(msg))
-                                       {
-                                               /*
-                                                * Bogus message length implies that we got out of
-                                                * sync with the buffer process somehow. Abort so that
-                                                * we can restart both processes.
-                                                */
-                                               ereport(ERROR,
-                                                         (errmsg("invalid statistics message length")));
-                                       }
-                               }
-                       }
+                       if (len < sizeof(PgStat_MsgHdr))
+                               continue;
 
                        /*
-                        * EOF on the pipe implies that the buffer process exited. Fall
-                        * out of outer loop.
+                        * The received length must match the length in the header
                         */
-                       if (pipeEOF)
-                               break;
+                       if (msg.msg_hdr.m_size != len)
+                               continue;
 
                        /*
-                        * Distribute the message to the specific function handling it.
+                        * O.K. - we accept this message.  Process it.
                         */
                        switch (msg.msg_hdr.m_type)
                        {
@@ -1869,32 +1746,32 @@ PgstatCollectorMain(int argc, char *argv[])
                                        break;
 
                                case PGSTAT_MTYPE_TABSTAT:
-                                       pgstat_recv_tabstat((PgStat_MsgTabstat *) &msg, nread);
+                                       pgstat_recv_tabstat((PgStat_MsgTabstat *) &msg, len);
                                        break;
 
                                case PGSTAT_MTYPE_TABPURGE:
-                                       pgstat_recv_tabpurge((PgStat_MsgTabpurge *) &msg, nread);
+                                       pgstat_recv_tabpurge((PgStat_MsgTabpurge *) &msg, len);
                                        break;
 
                                case PGSTAT_MTYPE_DROPDB:
-                                       pgstat_recv_dropdb((PgStat_MsgDropdb *) &msg, nread);
+                                       pgstat_recv_dropdb((PgStat_MsgDropdb *) &msg, len);
                                        break;
 
                                case PGSTAT_MTYPE_RESETCOUNTER:
                                        pgstat_recv_resetcounter((PgStat_MsgResetcounter *) &msg,
-                                                                                        nread);
+                                                                                        len);
                                        break;
 
                                case PGSTAT_MTYPE_AUTOVAC_START:
-                                       pgstat_recv_autovac((PgStat_MsgAutovacStart *) &msg, nread);
+                                       pgstat_recv_autovac((PgStat_MsgAutovacStart *) &msg, len);
                                        break;
 
                                case PGSTAT_MTYPE_VACUUM:
-                                       pgstat_recv_vacuum((PgStat_MsgVacuum *) &msg, nread);
+                                       pgstat_recv_vacuum((PgStat_MsgVacuum *) &msg, len);
                                        break;
 
                                case PGSTAT_MTYPE_ANALYZE:
-                                       pgstat_recv_analyze((PgStat_MsgAnalyze *) &msg, nread);
+                                       pgstat_recv_analyze((PgStat_MsgAnalyze *) &msg, len);
                                        break;
 
                                default:
@@ -1908,284 +1785,44 @@ PgstatCollectorMain(int argc, char *argv[])
                         */
                        if (need_timer)
                        {
-                               if (setitimer(ITIMER_REAL, &timeout, NULL))
+                               if (setitimer(ITIMER_REAL, &write_timeout, NULL))
                                        ereport(ERROR,
                                                  (errmsg("could not set statistics collector timer: %m")));
                                need_timer = false;
                        }
                }
-
-               /*
-                * Note that we do NOT check for postmaster exit inside the loop; only
-                * EOF on the buffer pipe causes us to fall out.  This ensures we
-                * don't exit prematurely if there are still a few messages in the
-                * buffer or pipe at postmaster shutdown.
-                */
-       }
-
-       /*
-        * Okay, we saw EOF on the buffer pipe, so there are no more messages to
-        * process.  If the buffer process quit because of postmaster shutdown, we
-        * want to save the final stats to reuse at next startup. But if the
-        * buffer process failed, it seems best not to (there may even now be a
-        * new collector firing up, and we don't want it to read a
-        * partially-rewritten stats file).
-        */
-       if (!PostmasterIsAlive(false))
-               pgstat_write_statsfile();
-}
-
-
-/* SIGALRM signal handler for collector process */
-static void
-force_statwrite(SIGNAL_ARGS)
-{
-       need_statwrite = true;
-}
-
-
-/* ----------
- * pgstat_recvbuffer() -
- *
- *     This is the body of the separate buffering process. Its only
- *     purpose is to receive messages from the UDP socket as fast as
- *     possible and forward them over a pipe into the collector itself.
- *     If the collector is slow to absorb messages, they are buffered here.
- * ----------
- */
-static void
-pgstat_recvbuffer(void)
-{
-       fd_set          rfds;
-       fd_set          wfds;
-       struct timeval timeout;
-       int                     writePipe = pgStatPipe[1];
-       int                     maxfd;
-       int                     len;
-       int                     xfr;
-       int                     frm;
-       PgStat_Msg      input_buffer;
-       char       *msgbuffer;
-       int                     msg_send = 0;   /* next send index in buffer */
-       int                     msg_recv = 0;   /* next receive index */
-       int                     msg_have = 0;   /* number of bytes stored */
-       bool            overflow = false;
-
-       /*
-        * Identify myself via ps
-        */
-       init_ps_display("stats buffer process", "", "", "");
-
-       /*
-        * We want to die if our child collector process does.  There are two ways
-        * we might notice that it has died: receive SIGCHLD, or get a write
-        * failure on the pipe leading to the child.  We can set SIGPIPE to kill
-        * us here.  Our SIGCHLD handler was already set up before we forked (must
-        * do it that way, else it's a race condition).
-        */
-       pqsignal(SIGPIPE, SIG_DFL);
-       PG_SETMASK(&UnBlockSig);
-
-       /*
-        * Set the write pipe to nonblock mode, so that we cannot block when the
-        * collector falls behind.
-        */
-       if (!pg_set_noblock(writePipe))
-               ereport(ERROR,
-                               (errcode_for_socket_access(),
-                                errmsg("could not set statistics collector pipe to nonblocking mode: %m")));
-
-       /*
-        * Allocate the message buffer
-        */
-       msgbuffer = (char *) palloc(PGSTAT_RECVBUFFERSZ);
-
-       /*
-        * Loop forever
-        */
-       for (;;)
-       {
-               FD_ZERO(&rfds);
-               FD_ZERO(&wfds);
-               maxfd = -1;
-
-               /*
-                * As long as we have buffer space we add the socket to the read
-                * descriptor set.
-                */
-               if (msg_have <= (int) (PGSTAT_RECVBUFFERSZ - sizeof(PgStat_Msg)))
-               {
-                       FD_SET(pgStatSock, &rfds);
-                       maxfd = pgStatSock;
-                       overflow = false;
-               }
                else
                {
-                       if (!overflow)
-                       {
-                               ereport(LOG,
-                                               (errmsg("statistics buffer is full")));
-                               overflow = true;
-                       }
-               }
-
-               /*
-                * If we have messages to write out, we add the pipe to the write
-                * descriptor set.
-                */
-               if (msg_have > 0)
-               {
-                       FD_SET(writePipe, &wfds);
-                       if (writePipe > maxfd)
-                               maxfd = writePipe;
-               }
-
-               /*
-                * Wait for some work to do; but not for more than 10 seconds. (This
-                * determines how quickly we will shut down after an ungraceful
-                * postmaster termination; so it needn't be very fast.)
-                *
-                * struct timeout is modified by select() on some operating systems,
-                * so re-fill it each time.
-                */
-               timeout.tv_sec = 10;
-               timeout.tv_usec = 0;
-
-               if (select(maxfd + 1, &rfds, &wfds, NULL, &timeout) < 0)
-               {
-                       if (errno == EINTR)
-                               continue;
-                       ereport(ERROR,
-                                       (errcode_for_socket_access(),
-                                        errmsg("select() failed in statistics buffer: %m")));
-               }
-
-               /*
-                * If there is a message on the socket, read it and check for
-                * validity.
-                */
-               if (FD_ISSET(pgStatSock, &rfds))
-               {
-                       len = recv(pgStatSock, (char *) &input_buffer,
-                                          sizeof(PgStat_Msg), 0);
-                       if (len < 0)
-                               ereport(ERROR,
-                                               (errcode_for_socket_access(),
-                                                errmsg("could not read statistics message: %m")));
-
-                       /*
-                        * We ignore messages that are smaller than our common header
-                        */
-                       if (len < sizeof(PgStat_MsgHdr))
-                               continue;
-
                        /*
-                        * The received length must match the length in the header
+                        * We can only get here if the select/poll timeout elapsed.
+                        * Check for postmaster death.
                         */
-                       if (input_buffer.msg_hdr.m_size != len)
-                               continue;
-
-                       /*
-                        * O.K. - we accept this message.  Copy it to the circular
-                        * msgbuffer.
-                        */
-                       frm = 0;
-                       while (len > 0)
-                       {
-                               xfr = PGSTAT_RECVBUFFERSZ - msg_recv;
-                               if (xfr > len)
-                                       xfr = len;
-                               Assert(xfr > 0);
-                               memcpy(msgbuffer + msg_recv,
-                                          ((char *) &input_buffer) + frm,
-                                          xfr);
-                               msg_recv += xfr;
-                               if (msg_recv == PGSTAT_RECVBUFFERSZ)
-                                       msg_recv = 0;
-                               msg_have += xfr;
-                               frm += xfr;
-                               len -= xfr;
-                       }
-               }
-
-               /*
-                * If the collector is ready to receive, write some data into his
-                * pipe.  We may or may not be able to write all that we have.
-                *
-                * NOTE: if what we have is less than PIPE_BUF bytes but more than the
-                * space available in the pipe buffer, most kernels will refuse to
-                * write any of it, and will return EAGAIN.  This means we will
-                * busy-loop until the situation changes (either because the collector
-                * caught up, or because more data arrives so that we have more than
-                * PIPE_BUF bytes buffered).  This is not good, but is there any way
-                * around it?  We have no way to tell when the collector has caught
-                * up...
-                */
-               if (FD_ISSET(writePipe, &wfds))
-               {
-                       xfr = PGSTAT_RECVBUFFERSZ - msg_send;
-                       if (xfr > msg_have)
-                               xfr = msg_have;
-                       Assert(xfr > 0);
-                       len = pipewrite(writePipe, msgbuffer + msg_send, xfr);
-                       if (len < 0)
-                       {
-                               if (errno == EINTR || errno == EAGAIN)
-                                       continue;       /* not enough space in pipe */
-                               ereport(ERROR,
-                                               (errcode_for_socket_access(),
-                               errmsg("could not write to statistics collector pipe: %m")));
-                       }
-                       /* NB: len < xfr is okay */
-                       msg_send += len;
-                       if (msg_send == PGSTAT_RECVBUFFERSZ)
-                               msg_send = 0;
-                       msg_have -= len;
+                       if (!PostmasterIsAlive(true))
+                               break;
                }
+       } /* end of message-processing loop */
 
-               /*
-                * Make sure we forwarded all messages before we check for postmaster
-                * termination.
-                */
-               if (msg_have != 0 || FD_ISSET(pgStatSock, &rfds))
-                       continue;
+       /*
+        * Save the final stats to reuse at next startup.
+        */
+       pgstat_write_statsfile();
 
-               /*
-                * If the postmaster has terminated, we die too.  (This is no longer
-                * the normal exit path, however.)
-                */
-               if (!PostmasterIsAlive(true))
-                       exit(0);
-       }
+       exit(0);
 }
 
-/* SIGQUIT signal handler for buffer process */
+
+/* SIGQUIT signal handler for collector process */
 static void
 pgstat_exit(SIGNAL_ARGS)
 {
-       /*
-        * For now, we just nail the doors shut and get out of town.  It might be
-        * cleaner to allow any pending messages to be sent, but that creates a
-        * tradeoff against speed of exit.
-        */
-
-       /*
-        * If running in bufferer, kill our collector as well. On some broken
-        * win32 systems, it does not shut down automatically because of issues
-        * with socket inheritance.  XXX so why not fix the socket inheritance...
-        */
-#ifdef WIN32
-       if (pgStatCollectorPid > 0)
-               kill(pgStatCollectorPid, SIGQUIT);
-#endif
-       exit(0);
+       need_exit = true;
 }
 
-/* SIGCHLD signal handler for buffer process */
+/* SIGALRM signal handler for collector process */
 static void
-pgstat_die(SIGNAL_ARGS)
+force_statwrite(SIGNAL_ARGS)
 {
-       exit(1);
+       need_statwrite = true;
 }
 
 
index faf124587da027a0b05b0614beb9f13c456371a1..21f651085b5b6d115d18466e8fd96a1d48ac65cf 100644 (file)
@@ -37,7 +37,7 @@
  *
  *
  * IDENTIFICATION
- *       $PostgreSQL: pgsql/src/backend/postmaster/postmaster.c,v 1.489 2006/06/27 22:16:43 momjian Exp $
+ *       $PostgreSQL: pgsql/src/backend/postmaster/postmaster.c,v 1.490 2006/06/29 20:00:08 tgl Exp $
  *
  * NOTES
  *
@@ -331,8 +331,6 @@ typedef struct
        PROC_HDR   *ProcGlobal;
        PGPROC     *DummyProcs;
        InheritableSocket pgStatSock;
-       InheritableSocket pgStatPipe0;
-       InheritableSocket pgStatPipe1;
        pid_t           PostmasterPid;
        TimestampTz PgStartTime;
 #ifdef WIN32
@@ -2352,7 +2350,7 @@ HandleChildCrash(int pid, int exitstatus, const char *procname)
                kill(PgArchPID, SIGQUIT);
        }
 
-       /* Force a power-cycle of the pgstat processes too */
+       /* Force a power-cycle of the pgstat process too */
        /* (Shouldn't be necessary, but just for luck) */
        if (PgStatPID != 0 && !FatalError)
        {
@@ -3324,25 +3322,13 @@ SubPostmasterMain(int argc, char *argv[])
                PgArchiverMain(argc, argv);
                proc_exit(0);
        }
-       if (strcmp(argv[1], "--forkbuf") == 0)
+       if (strcmp(argv[1], "--forkcol") == 0)
        {
                /* Close the postmaster's sockets */
                ClosePostmasterPorts(false);
 
                /* Do not want to attach to shared memory */
 
-               PgstatBufferMain(argc, argv);
-               proc_exit(0);
-       }
-       if (strcmp(argv[1], "--forkcol") == 0)
-       {
-               /*
-                * Do NOT close postmaster sockets here, because we are forking from
-                * pgstat buffer process, which already did it.
-                */
-
-               /* Do not want to attach to shared memory */
-
                PgstatCollectorMain(argc, argv);
                proc_exit(0);
        }
@@ -3679,7 +3665,6 @@ extern slock_t *ProcStructLock;
 extern PROC_HDR *ProcGlobal;
 extern PGPROC *DummyProcs;
 extern int     pgStatSock;
-extern int     pgStatPipe[2];
 
 #ifndef WIN32
 #define write_inheritable_socket(dest, src, childpid) (*(dest) = (src))
@@ -3723,8 +3708,6 @@ save_backend_variables(BackendParameters * param, Port *port,
        param->ProcGlobal = ProcGlobal;
        param->DummyProcs = DummyProcs;
        write_inheritable_socket(&param->pgStatSock, pgStatSock, childPid);
-       write_inheritable_socket(&param->pgStatPipe0, pgStatPipe[0], childPid);
-       write_inheritable_socket(&param->pgStatPipe1, pgStatPipe[1], childPid);
 
        param->PostmasterPid = PostmasterPid;
        param->PgStartTime = PgStartTime;
@@ -3928,8 +3911,6 @@ restore_backend_variables(BackendParameters * param, Port *port)
        ProcGlobal = param->ProcGlobal;
        DummyProcs = param->DummyProcs;
        read_inheritable_socket(&pgStatSock, &param->pgStatSock);
-       read_inheritable_socket(&pgStatPipe[0], &param->pgStatPipe0);
-       read_inheritable_socket(&pgStatPipe[1], &param->pgStatPipe1);
 
        PostmasterPid = param->PostmasterPid;
        PgStartTime = param->PgStartTime;
index 83a45d7217f6c75e3073d4a1b5a5046a60e0fc82..72e542f16c8a7bdeb19e4e761564d764d6a41385 100644 (file)
@@ -5,7 +5,7 @@
  *
  *     Copyright (c) 2001-2006, PostgreSQL Global Development Group
  *
- *     $PostgreSQL: pgsql/src/include/pgstat.h,v 1.47 2006/06/19 01:51:21 tgl Exp $
+ *     $PostgreSQL: pgsql/src/include/pgstat.h,v 1.48 2006/06/29 20:00:08 tgl Exp $
  * ----------
  */
 #ifndef PGSTAT_H
@@ -362,7 +362,6 @@ extern int  pgstat_start(void);
 extern void pgstat_reset_all(void);
 
 #ifdef EXEC_BACKEND
-extern void PgstatBufferMain(int argc, char *argv[]);
 extern void PgstatCollectorMain(int argc, char *argv[]);
 #endif