]> granicus.if.org Git - postgresql/commitdiff
Improve autovacuum launcher's ability to detect a problem in worker startup,
authorAlvaro Herrera <alvherre@alvh.no-ip.org>
Mon, 25 Jun 2007 16:09:03 +0000 (16:09 +0000)
committerAlvaro Herrera <alvherre@alvh.no-ip.org>
Mon, 25 Jun 2007 16:09:03 +0000 (16:09 +0000)
by having the postmaster signal it when certain failures occur.  This requires
the postmaster setting a flag in shared memory, but should be as safe as the
pmsignal.c code is.

Also make sure the launcher honor's a postgresql.conf change turning it off
on SIGHUP.

src/backend/postmaster/autovacuum.c
src/backend/postmaster/postmaster.c
src/include/postmaster/autovacuum.h

index 46eb38e8e4701516e262807ff8aa15f36063c14a..bc2dfee28fc5b46aae45c79e69a3e360f976dd5c 100644 (file)
@@ -4,13 +4,58 @@
  *
  * PostgreSQL Integrated Autovacuum Daemon
  *
+ * The autovacuum system is structured in two different kinds of processes: the
+ * autovacuum launcher and the autovacuum worker.  The launcher is an
+ * always-running process, started by the postmaster when the autovacuum GUC
+ * parameter is set.  The launcher schedules autovacuum workers to be started
+ * when appropriate.  The workers are the processes which execute the actual
+ * vacuuming; they connect to a database as determined in the launcher, and
+ * once connected they examine the catalogs to select the tables to vacuum.
+ *
+ * The autovacuum launcher cannot start the worker processes by itself,
+ * because doing so would cause robustness issues (namely, failure to shut
+ * them down on exceptional conditions, and also, since the launcher is
+ * connected to shared memory and is thus subject to corruption there, it is
+ * not as robust as the postmaster).  So it leaves that task to the postmaster.
+ *
+ * There is an autovacuum shared memory area, where the launcher stores
+ * information about the database it wants vacuumed.  When it wants a new
+ * worker to start, it sets a flag in shared memory and sends a signal to the
+ * postmaster.  Then postmaster knows nothing more than it must start a worker;
+ * so it forks a new child, which turns into a worker.  This new process
+ * connects to shared memory, and there it can inspect the information that the
+ * launcher has set up.
+ *
+ * If the fork() call fails in the postmaster, it sets a flag in the shared
+ * memory area, and sends a signal to the launcher.  The launcher, upon
+ * noticing the flag, can try starting the worker again by resending the
+ * signal.  Note that the failure can only be transient (fork failure due to
+ * high load, memory pressure, too many processes, etc); more permanent
+ * problems, like failure to connect to a database, are detected later in the
+ * worker and dealt with just by having the worker exit normally.  The launcher
+ * will launch a new worker again later, per schedule.
+ *
+ * When the worker is done vacuuming it sends SIGUSR1 to the launcher.  The
+ * launcher then wakes up and is able to launch another worker, if the schedule
+ * is so tight that a new worker is needed immediately.  At this time the
+ * launcher can also balance the settings for the various remaining workers'
+ * cost-based vacuum delay feature.
+ *
+ * Note that there can be more than one worker in a database concurrently.
+ * They will store the table they are currently vacuuming in shared memory, so
+ * that other workers avoid being blocked waiting for the vacuum lock for that
+ * table.  They will also reload the pgstats data just before vacuuming each
+ * table, to avoid vacuuming a table that was just finished being vacuumed by
+ * another worker and thus is no longer noted in shared memory.  However,
+ * there is a window (caused by pgstat delay) on which a worker may choose a
+ * table that was already vacuumed; this is a bug in the current design.
  *
  * Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  *
  * IDENTIFICATION
- *       $PostgreSQL: pgsql/src/backend/postmaster/autovacuum.c,v 1.50 2007/06/13 21:24:55 alvherre Exp $
+ *       $PostgreSQL: pgsql/src/backend/postmaster/autovacuum.c,v 1.51 2007/06/25 16:09:03 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -158,28 +203,40 @@ typedef struct WorkerInfoData
 
 typedef struct WorkerInfoData *WorkerInfo;
 
+/*
+ * Possible signals received by the launcher from remote processes.  These are
+ * stored atomically in shared memory so that other processes can set them
+ * without locking.
+ */
+typedef enum 
+{
+       AutoVacForkFailed,      /* failed trying to start a worker */
+       AutoVacRebalance,       /* rebalance the cost limits */
+       AutoVacNumSignals = AutoVacRebalance    /* must be last */
+} AutoVacuumSignal;
+
 /*-------------
  * The main autovacuum shmem struct.  On shared memory we store this main
  * struct and the array of WorkerInfo structs.  This struct keeps:
  *
+ * av_signal           set by other processes to indicate various conditions
  * av_launcherpid      the PID of the autovacuum launcher
  * av_freeWorkers      the WorkerInfo freelist
  * av_runningWorkers the WorkerInfo non-free queue
  * av_startingWorker pointer to WorkerInfo currently being started (cleared by
  *                                     the worker itself as soon as it's up and running)
- * av_rebalance                true when a worker determines that cost limits must be
- *                                     rebalanced
  *
- * This struct is protected by AutovacuumLock.
+ * This struct is protected by AutovacuumLock, except for av_signal and parts
+ * of the worker list (see above).
  *-------------
  */
 typedef struct
 {
+       sig_atomic_t    av_signal[AutoVacNumSignals];
        pid_t                   av_launcherpid;
        SHMEM_OFFSET    av_freeWorkers;
        SHM_QUEUE               av_runningWorkers;
        SHMEM_OFFSET    av_startingWorker;
-       bool                    av_rebalance;
 } AutoVacuumShmemStruct;
 
 static AutoVacuumShmemStruct *AutoVacuumShmem;
@@ -316,29 +373,6 @@ StartAutoVacLauncher(void)
 
 /*
  * Main loop for the autovacuum launcher process.
- *
- * The signalling between launcher and worker is as follows:
- *
- * When the worker has finished starting up, it stores its PID in wi_workerpid
- * and sends a SIGUSR1 signal to the launcher.  The launcher then knows that
- * the postmaster is ready to start a new worker.  We do it this way because
- * otherwise we risk calling SendPostmasterSignal() when the postmaster hasn't
- * yet processed the last one, in which case the second signal would be lost.
- * This is only useful when two workers need to be started close to one
- * another, which should be rare but it's possible.
- *
- * When a worker exits, it resets the WorkerInfo struct and puts it back into
- * the free list.  If there is no free worker slot, it will also signal the
- * launcher, which then wakes up and can launch a new worker if it needs to.
- * Note that we only need to do it when there's no free worker slot, because
- * otherwise there is no need -- the launcher would be awakened normally per
- * schedule.
- *
- * There is a potential problem if, for some reason, a worker starts and is not
- * able to bootstrap itself correctly.  To prevent this situation from starving
- * the whole system, the launcher checks the launch time of the "starting
- * worker".  If it's too old (older than autovacuum_naptime seconds), it resets
- * the worker entry and puts it back into the free list.
  */
 NON_EXEC_STATIC void
 AutoVacLauncherMain(int argc, char *argv[])
@@ -494,8 +528,9 @@ AutoVacLauncherMain(int argc, char *argv[])
        for (;;)
        {
                struct timeval nap;
-               bool    can_launch;
                TimestampTz current_time = 0;
+               bool    can_launch;
+               Dlelem *elem;
 
                /*
                 * Emergency bailout if postmaster has died.  This is to avoid the
@@ -547,6 +582,10 @@ AutoVacLauncherMain(int argc, char *argv[])
                        got_SIGHUP = false;
                        ProcessConfigFile(PGC_SIGHUP);
 
+                       /* shutdown requested in config file */
+                       if (!autovacuum_start_daemon)
+                               break;
+
                        /* rebalance in case the default cost parameters changed */
                        LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
                        autovac_balance_cost();
@@ -556,51 +595,77 @@ AutoVacLauncherMain(int argc, char *argv[])
                        rebuild_database_list(InvalidOid);
                }
 
-               /* a worker started up or finished */
+               /*
+                * a worker finished, or postmaster signalled failure to start a
+                * worker
+                */
                if (got_SIGUSR1)
                {
                        got_SIGUSR1 = false;
 
                        /* rebalance cost limits, if needed */
-                       if (AutoVacuumShmem->av_rebalance)
+                       if (AutoVacuumShmem->av_signal[AutoVacRebalance])
                        {
                                LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
-                               AutoVacuumShmem->av_rebalance = false;
+                               AutoVacuumShmem->av_signal[AutoVacRebalance] = false;
                                autovac_balance_cost();
                                LWLockRelease(AutovacuumLock);
                        }
+
+                       if (AutoVacuumShmem->av_signal[AutoVacForkFailed])
+                       {
+                               /*
+                                * If the postmaster failed to start a new worker, we sleep
+                                * for a little while and resend the signal.  The new worker's
+                                * state is still in memory, so this is sufficient.  After
+                                * that, we restart the main loop.
+                                *
+                                * XXX should we put a limit to the number of times we retry?
+                                * I don't think it makes much sense, because a future start
+                                * of a worker will continue to fail in the same way.
+                                */
+                               AutoVacuumShmem->av_signal[AutoVacForkFailed] = false;
+                               pg_usleep(100000L);     /* 100ms */
+                               SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_WORKER);
+                               continue;
+                       }
                }
 
                /*
                 * There are some conditions that we need to check before trying to
                 * start a launcher.  First, we need to make sure that there is a
                 * launcher slot available.  Second, we need to make sure that no other
-                * worker is still starting up.
+                * worker failed while starting up.
                 */
 
+               current_time = GetCurrentTimestamp();
                LWLockAcquire(AutovacuumLock, LW_SHARED);
 
                can_launch = (AutoVacuumShmem->av_freeWorkers != INVALID_OFFSET);
 
-               if (can_launch && AutoVacuumShmem->av_startingWorker != INVALID_OFFSET)
+               if (AutoVacuumShmem->av_startingWorker != INVALID_OFFSET)
                {
-                       WorkerInfo worker = (WorkerInfo) MAKE_PTR(AutoVacuumShmem->av_startingWorker);
+                       int             waittime;
 
-                       if (current_time == 0)
-                               current_time = GetCurrentTimestamp();
+                       WorkerInfo worker = (WorkerInfo) MAKE_PTR(AutoVacuumShmem->av_startingWorker);
 
                        /*
                         * We can't launch another worker when another one is still
-                        * starting up, so just sleep for a bit more; that worker will wake
-                        * us up again as soon as it's ready.  We will only wait
-                        * autovacuum_naptime seconds for this to happen however.  Note
-                        * that failure to connect to a particular database is not a
-                        * problem here, because the worker removes itself from the
-                        * startingWorker pointer before trying to connect; only low-level
-                        * problems, like fork() failure, can get us here.
+                        * starting up (or failed while doing so), so just sleep for a bit
+                        * more; that worker will wake us up again as soon as it's ready.
+                        * We will only wait autovacuum_naptime seconds (up to a maximum of
+                        * 60 seconds) for this to happen however.  Note that failure to
+                        * connect to a particular database is not a problem here, because
+                        * the worker removes itself from the startingWorker pointer before
+                        * trying to connect.  Problems detected by the postmaster (like
+                        * fork() failure) are also reported and handled differently.  The
+                        * only problems that may cause this code to fire are errors in the
+                        * earlier sections of AutoVacWorkerMain, before the worker removes
+                        * the WorkerInfo from the startingWorker pointer.
                         */
+                       waittime = Min(autovacuum_naptime, 60) * 1000;
                        if (TimestampDifferenceExceeds(worker->wi_launchtime, current_time,
-                                                                                  autovacuum_naptime * 1000))
+                                                                                  waittime))
                        {
                                LWLockRelease(AutovacuumLock);
                                LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
@@ -620,55 +685,43 @@ AutoVacLauncherMain(int argc, char *argv[])
                                        worker->wi_links.next = AutoVacuumShmem->av_freeWorkers;
                                        AutoVacuumShmem->av_freeWorkers = MAKE_OFFSET(worker);
                                        AutoVacuumShmem->av_startingWorker = INVALID_OFFSET;
+                                       elog(WARNING, "worker took too long to start; cancelled");
                                }
                        }
                        else
-                       {
-                               /*
-                                * maybe the postmaster neglected this start signal --
-                                * resend it.  Note: the constraints in
-                                * launcher_determine_sleep keep us from delivering signals too
-                                * quickly (at most once every 100ms).
-                                */
-                               SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_WORKER);
                                can_launch = false;
-                       }
                }
                LWLockRelease(AutovacuumLock);          /* either shared or exclusive */
 
-               if (can_launch)
-               {
-                       Dlelem     *elem;
-
-                       elem = DLGetTail(DatabaseList);
+               /* if we can't do anything, just go back to sleep */
+               if (!can_launch)
+                       continue;
 
-                       if (current_time == 0)
-                               current_time = GetCurrentTimestamp();
+               /* We're OK to start a new worker */
 
-                       if (elem != NULL)
-                       {
-                               avl_dbase *avdb = DLE_VAL(elem);
+               elem = DLGetTail(DatabaseList);
+               if (elem != NULL)
+               {
+                       avl_dbase *avdb = DLE_VAL(elem);
 
-                               /*
-                                * launch a worker if next_worker is right now or it is in the
-                                * past
-                                */
-                               if (TimestampDifferenceExceeds(avdb->adl_next_worker,
-                                                                                          current_time, 0))
-                                       launch_worker(current_time);
-                       }
-                       else
-                       {
-                               /*
-                                * Special case when the list is empty: start a worker right
-                                * away.  This covers the initial case, when no database is in
-                                * pgstats (thus the list is empty).  Note that the constraints
-                                * in launcher_determine_sleep keep us from starting workers
-                                * too quickly (at most once every autovacuum_naptime when the
-                                * list is empty).
-                                */
+                       /*
+                        * launch a worker if next_worker is right now or it is in the past
+                        */
+                       if (TimestampDifferenceExceeds(avdb->adl_next_worker,
+                                                                                  current_time, 0))
                                launch_worker(current_time);
-                       }
+               }
+               else
+               {
+                       /*
+                        * Special case when the list is empty: start a worker right away.
+                        * This covers the initial case, when no database is in pgstats
+                        * (thus the list is empty).  Note that the constraints in
+                        * launcher_determine_sleep keep us from starting workers too
+                        * quickly (at most once every autovacuum_naptime when the list is
+                        * empty).
+                        */
+                       launch_worker(current_time);
                }
        }
 
@@ -1197,6 +1250,17 @@ launch_worker(TimestampTz now)
        }
 }
 
+/*
+ * Called from postmaster to signal a failure to fork a process to become
+ * worker.  The postmaster should kill(SIGUSR1) the launcher shortly
+ * after calling this function.
+ */
+void
+AutoVacWorkerFailed(void)
+{
+       AutoVacuumShmem->av_signal[AutoVacForkFailed] = true;
+}
+
 /* SIGHUP: set flag to re-read config file at next convenient time */
 static void
 avl_sighup_handler(SIGNAL_ARGS)
@@ -1435,8 +1499,9 @@ AutoVacWorkerMain(int argc, char *argv[])
        LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
 
        /*
-        * beware of startingWorker being INVALID; this could happen if the
-        * launcher thinks we've taking too long to start.
+        * beware of startingWorker being INVALID; this should normally not happen,
+        * but if a worker fails after forking and before this, the launcher might
+        * have decided to remove it from the queue and start again.
         */
        if (AutoVacuumShmem->av_startingWorker != INVALID_OFFSET)
        {
@@ -1463,6 +1528,7 @@ AutoVacWorkerMain(int argc, char *argv[])
        else
        {
                /* no worker entry for me, go away */
+               elog(WARNING, "autovacuum worker started without a worker entry");
                dbid = InvalidOid;
                LWLockRelease(AutovacuumLock);
        }
@@ -1525,19 +1591,18 @@ FreeWorkerInfo(int code, Datum arg)
                LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
 
                /*
-                * If this worker shuts down when there is no free worker slot, wake
-                * the launcher up so that he can launch a new worker immediately if
-                * required.  We only save the launcher's PID in local memory here --
-                * the actual signal will be sent when the PGPROC is recycled, because
-                * that is when the new worker can actually be launched.
+                * Wake the launcher up so that he can launch a new worker immediately
+                * if required.  We only save the launcher's PID in local memory here;
+                * the actual signal will be sent when the PGPROC is recycled.  Note
+                * that we always do this, so that the launcher can rebalance the cost
+                * limit setting of the remaining workers.
                 *
                 * We somewhat ignore the risk that the launcher changes its PID
                 * between we reading it and the actual kill; we expect ProcKill to be
                 * called shortly after us, and we assume that PIDs are not reused too
                 * quickly after a process exits.
                 */
-               if (AutoVacuumShmem->av_freeWorkers == INVALID_OFFSET)
-                       AutovacuumLauncherPid = AutoVacuumShmem->av_launcherpid;
+               AutovacuumLauncherPid = AutoVacuumShmem->av_launcherpid;
 
                SHMQueueDelete(&MyWorkerInfo->wi_links);
                MyWorkerInfo->wi_links.next = AutoVacuumShmem->av_freeWorkers;
@@ -1556,7 +1621,7 @@ FreeWorkerInfo(int code, Datum arg)
                 * now that we're inactive, cause a rebalancing of the surviving
                 * workers
                 */
-               AutoVacuumShmem->av_rebalance = true;
+               AutoVacuumShmem->av_signal[AutoVacRebalance] = true;
                LWLockRelease(AutovacuumLock);
        }
 }
index 385185eddb45b0da8f111e7d2d802342462ba97c..cb060878a92a8e19a70925e180a9edfad3d73b42 100644 (file)
@@ -37,7 +37,7 @@
  *
  *
  * IDENTIFICATION
- *       $PostgreSQL: pgsql/src/backend/postmaster/postmaster.c,v 1.527 2007/03/22 19:53:30 momjian Exp $
+ *       $PostgreSQL: pgsql/src/backend/postmaster/postmaster.c,v 1.528 2007/06/25 16:09:03 alvherre Exp $
  *
  * NOTES
  *
@@ -3830,32 +3830,35 @@ StartAutovacuumWorker(void)
                return;
 
        bn = (Backend *) malloc(sizeof(Backend));
-       if (!bn)
+       if (bn)
        {
-               ereport(LOG,
-                               (errcode(ERRCODE_OUT_OF_MEMORY),
-                                errmsg("out of memory")));
-               return;
-       }
-
-       bn->pid = StartAutoVacWorker();
-       bn->is_autovacuum = true;
-       /* we don't need a cancel key */
+               bn->pid = StartAutoVacWorker();
+               bn->is_autovacuum = true;
+               /* we don't need a cancel key */
 
-       if (bn->pid > 0)
-       {
-               DLAddHead(BackendList, DLNewElem(bn));
+               if (bn->pid > 0)
+               {
+                       DLAddHead(BackendList, DLNewElem(bn));
 #ifdef EXEC_BACKEND
-               ShmemBackendArrayAdd(bn);
+                       ShmemBackendArrayAdd(bn);
 #endif
-       }
-       else
-       {
-               /* not much we can do */
-               ereport(LOG,
-                               (errmsg("could not fork new process for autovacuum: %m")));
+                       /* all OK */
+                       return;
+               }
+
+               /*
+                * fork failed, fall through to report -- actual error message was
+                * logged by StartAutoVacWorker
+                */
                free(bn);
        }
+       else
+               elog(LOG, "out of memory");
+
+       /* report the failure to the launcher */
+       AutoVacWorkerFailed();
+       if (AutoVacPID != 0)
+               kill(AutoVacPID, SIGUSR1);
 }
 
 /*
index bc464de14af12f296f7cdc486930e45b81ef6cf8..27a982aafc5b21f4f9683c63162c499132683067 100644 (file)
@@ -7,7 +7,7 @@
  * Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/postmaster/autovacuum.h,v 1.10 2007/04/18 16:44:18 alvherre Exp $
+ * $PostgreSQL: pgsql/src/include/postmaster/autovacuum.h,v 1.11 2007/06/25 16:09:03 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -42,6 +42,8 @@ extern bool IsAutoVacuumWorkerProcess(void);
 extern void autovac_init(void);
 extern int     StartAutoVacLauncher(void);
 extern int     StartAutoVacWorker(void);
+/* called from postmaster when a worker could not be forked */
+extern void AutoVacWorkerFailed(void);
 
 /* autovacuum cost-delay balancer */
 extern void AutoVacuumUpdateDelay(void);