diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 76865d975234beb02716f24ca471fb0052d29188..c9754581ec71689def586b6552f405e473d11e8a 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -6,7 +6,7 @@
  * Copyright (c) 2000-2007, PostgreSQL Global Development Group
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/access/transam/varsup.c,v 1.77 2007/01/05 22:19:23 momjian Exp $
+ *	  $PostgreSQL: pgsql/src/backend/access/transam/varsup.c,v 1.78 2007/02/15 23:23:22 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -72,7 +72,7 @@ GetNewTransactionId(bool isSubXact)
 		 * still gives plenty of chances before we get into real trouble.
 		 */
 		if (IsUnderPostmaster && (xid % 65536) == 0)
-			SendPostmasterSignal(PMSIGNAL_START_AUTOVAC);
+			SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_LAUNCHER);
 
 		if (IsUnderPostmaster &&
 		 TransactionIdFollowsOrEquals(xid, ShmemVariableCache->xidStopLimit))
@@ -286,7 +286,7 @@ SetTransactionIdLimit(TransactionId oldest_datfrozenxid,
 	 */
 	if (TransactionIdFollowsOrEquals(curXid, xidVacLimit) &&
 		IsUnderPostmaster)
-		SendPostmasterSignal(PMSIGNAL_START_AUTOVAC);
+		SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_LAUNCHER);
 
 	/* Give an immediate warning if past the wrap warn point */
 	if (TransactionIdFollowsOrEquals(curXid, xidWarnLimit))
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index be4a219ef0a467c54f17b6e578ee01492278175e..6e6efe744ba77e37fac49b182ec7f2ae74b32c1d 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -8,7 +8,7 @@
  * Portions Copyright (c) 1994, Regents of the University of California
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/bootstrap/bootstrap.c,v 1.230 2007/02/10 14:58:54 petere Exp $
+ *	  $PostgreSQL: pgsql/src/backend/bootstrap/bootstrap.c,v 1.231 2007/02/15 23:23:22 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -449,7 +449,7 @@ BootstrapMain(int argc, char *argv[])
 	 * Do backend-like initialization for bootstrap mode
 	 */
 	InitProcess();
-	(void) InitPostgres(dbname, NULL);
+	(void) InitPostgres(dbname, InvalidOid, NULL, NULL);
 
 	/*
 	 * In NOP mode, all we really want to do is create shared memory and
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index 5f9f787eeea4be7c7be98a0c2f349668ab0f165a..804897821ce976c1934d70f718debd4c01d854bd 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -13,7 +13,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.345 2007/02/05 04:22:18 tgl Exp $
+ *	  $PostgreSQL: pgsql/src/backend/commands/vacuum.c,v 1.346 2007/02/15 23:23:22 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -303,7 +303,7 @@ vacuum(VacuumStmt *vacstmt, List *relids)
 	 * Send info about dead objects to the statistics collector, unless we are
 	 * in autovacuum --- autovacuum.c does this for itself.
 	 */
-	if (vacstmt->vacuum && !IsAutoVacuumProcess())
+	if (vacstmt->vacuum && !IsAutoVacuumWorkerProcess())
 		pgstat_vacuum_tabstat();
 
 	/*
@@ -472,7 +472,7 @@ vacuum(VacuumStmt *vacstmt, List *relids)
 		ActiveSnapshot = CopySnapshot(GetTransactionSnapshot());
 	}
 
-	if (vacstmt->vacuum && !IsAutoVacuumProcess())
+	if (vacstmt->vacuum && !IsAutoVacuumWorkerProcess())
 	{
 		/*
 		 * Update pg_database.datfrozenxid, and truncate pg_clog if possible.
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 130ad45fd8be984bc564c018e2dc1dc6745bf109..672c3d5aa40c552b1be03c049509ebf64d7d1d5a 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -10,7 +10,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/postmaster/autovacuum.c,v 1.31 2007/01/16 13:28:56 alvherre Exp $
+ *	  $PostgreSQL: pgsql/src/backend/postmaster/autovacuum.c,v 1.32 2007/02/15 23:23:23 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -39,7 +39,9 @@
 #include "postmaster/postmaster.h"
 #include "storage/fd.h"
 #include "storage/ipc.h"
+#include "storage/pmsignal.h"
 #include "storage/proc.h"
+#include "storage/procarray.h"
 #include "storage/sinval.h"
 #include "tcop/tcopprot.h"
 #include "utils/flatfiles.h"
@@ -50,6 +52,9 @@
 #include "utils/syscache.h"
 
 
+static volatile sig_atomic_t got_SIGHUP = false;
+static volatile sig_atomic_t avlauncher_shutdown_request = false;
+
 /*
  * GUC parameters
  */
@@ -65,11 +70,8 @@ int			autovacuum_vac_cost_delay;
 int			autovacuum_vac_cost_limit;
 
 /* Flag to tell if we are in the autovacuum daemon process */
-static bool am_autovacuum = false;
-
-/* Last time autovac daemon started/stopped (only valid in postmaster) */
-static time_t last_autovac_start_time = 0;
-static time_t last_autovac_stop_time = 0;
+static bool am_autovacuum_launcher = false;
+static bool am_autovacuum_worker = false;
 
 /* Comparison point for determining whether freeze_max_age is exceeded */
 static TransactionId recentXid;
@@ -101,11 +103,21 @@ typedef struct autovac_table
 	int			vacuum_cost_limit;
 } autovac_table;
 
+typedef struct
+{
+	Oid		process_db;			/* OID of database to process */
+	int		worker_pid;			/* PID of the worker process, if any */
+} AutoVacuumShmemStruct;
+
+static AutoVacuumShmemStruct *AutoVacuumShmem;
 
 #ifdef EXEC_BACKEND
-static pid_t autovac_forkexec(void);
+static pid_t avlauncher_forkexec(void);
+static pid_t avworker_forkexec(void);
 #endif
-NON_EXEC_STATIC void AutoVacMain(int argc, char *argv[]);
+NON_EXEC_STATIC void AutoVacWorkerMain(int argc, char *argv[]);
+NON_EXEC_STATIC void AutoVacLauncherMain(int argc, char *argv[]);
+
 static void do_autovacuum(PgStat_StatDBEntry *dbentry);
 static List *autovac_get_database_list(void);
 static void test_rel_for_autovac(Oid relid, PgStat_StatTabEntry *tabentry,
@@ -116,47 +128,59 @@ static void test_rel_for_autovac(Oid relid, PgStat_StatTabEntry *tabentry,
 static void autovacuum_do_vac_analyze(Oid relid, bool dovacuum,
 						  bool doanalyze, int freeze_min_age);
 static void autovac_report_activity(VacuumStmt *vacstmt, Oid relid);
+static void avl_sighup_handler(SIGNAL_ARGS);
+static void avlauncher_shutdown(SIGNAL_ARGS);
+static void avl_quickdie(SIGNAL_ARGS);
 
 
+
+/********************************************************************
+ *                    AUTOVACUUM LAUNCHER CODE
+ ********************************************************************/
+
+#ifdef EXEC_BACKEND
 /*
- * Main entry point for autovacuum controller process.
+ * forkexec routine for the autovacuum launcher process.
  *
- * This code is heavily based on pgarch.c, q.v.
+ * Format up the arglist, then fork and exec.
  */
-int
-autovac_start(void)
+static pid_t
+avlauncher_forkexec(void)
 {
-	time_t		curtime;
-	pid_t		AutoVacPID;
+	char	   *av[10];
+	int			ac = 0;
 
-	/*
-	 * Do nothing if too soon since last autovacuum exit.  This limits how
-	 * often the daemon runs.  Since the time per iteration can be quite
-	 * variable, it seems more useful to measure/control the time since last
-	 * subprocess exit than since last subprocess launch.
-	 *
-	 * However, we *also* check the time since last subprocess launch; this
-	 * prevents thrashing under fork-failure conditions.
-	 *
-	 * Note that since we will be re-called from the postmaster main loop, we
-	 * will get another chance later if we do nothing now.
-	 *
-	 * XXX todo: implement sleep scale factor that existed in contrib code.
-	 */
+	av[ac++] = "postgres";
+	av[ac++] = "--forkavlauncher";
+	av[ac++] = NULL;			/* filled in by postmaster_forkexec */
+	av[ac] = NULL;
 
-	curtime = time(NULL);
-	if ((unsigned int) (curtime - last_autovac_stop_time) <
-		(unsigned int) autovacuum_naptime)
-		return 0;
+	Assert(ac < lengthof(av));
 
-	if ((unsigned int) (curtime - last_autovac_start_time) <
-		(unsigned int) autovacuum_naptime)
-		return 0;
+	return postmaster_forkexec(ac, av);
+}
 
-	last_autovac_start_time = curtime;
+/*
+ * We need this set from the outside, before InitProcess is called
+ */
+void
+AutovacuumLauncherIAm(void)
+{
+	am_autovacuum_launcher = true;
+}
+#endif
+
+/*
+ * Main entry point for autovacuum launcher process, to be called from the
+ * postmaster.
+ */
+int
+StartAutoVacLauncher(void)
+{
+	pid_t		AutoVacPID;
 
 #ifdef EXEC_BACKEND
-	switch ((AutoVacPID = autovac_forkexec()))
+	switch ((AutoVacPID = avlauncher_forkexec()))
 #else
 	switch ((AutoVacPID = fork_process()))
 #endif
@@ -175,7 +199,7 @@ autovac_start(void)
 			/* Lose the postmaster's on-exit routines */
 			on_exit_reset();
 
-			AutoVacMain(0, NULL);
+			AutoVacLauncherMain(0, NULL);
 			break;
 #endif
 		default:
@@ -187,28 +211,362 @@ autovac_start(void)
 }
 
 /*
- * autovac_stopped --- called by postmaster when subprocess exit is detected
+ * Main loop for the autovacuum launcher process.
  */
-void
-autovac_stopped(void)
+NON_EXEC_STATIC void
+AutoVacLauncherMain(int argc, char *argv[])
 {
-	last_autovac_stop_time = time(NULL);
+	sigjmp_buf	local_sigjmp_buf;
+	List	   *dblist;
+	bool		for_xid_wrap;
+	autovac_dbase *db;
+	MemoryContext	avlauncher_cxt;
+
+	/* we are a postmaster subprocess now */
+	IsUnderPostmaster = true;
+	am_autovacuum_launcher = true;
+
+	/* reset MyProcPid */
+	MyProcPid = getpid();
+
+	/* Identify myself via ps */
+	init_ps_display("autovacuum launcher process", "", "", "");
+
+	SetProcessingMode(InitProcessing);
+
+	/*
+	 * If possible, make this process a group leader, so that the postmaster
+	 * can signal any child processes too.  (autovacuum probably never has
+	 * any child processes, but for consistency we make all postmaster
+	 * child processes do this.)
+	 */
+#ifdef HAVE_SETSID
+	if (setsid() < 0)
+		elog(FATAL, "setsid() failed: %m");
+#endif
+
+	/*
+	 * Set up signal handlers.	Since this is a "dummy" process, it has
+	 * particular signal requirements -- no deadlock checker or sinval
+	 * catchup, for example.
+	 *
+	 * XXX It may be a good idea to receive signals when an avworker process
+	 * finishes.
+	 */
+	pqsignal(SIGHUP, avl_sighup_handler);
+
+	pqsignal(SIGINT, SIG_IGN);
+	pqsignal(SIGTERM, avlauncher_shutdown);
+	pqsignal(SIGQUIT, avl_quickdie);
+	pqsignal(SIGALRM, SIG_IGN);
+
+	pqsignal(SIGPIPE, SIG_IGN);
+	pqsignal(SIGUSR1, SIG_IGN);
+	/* We don't listen for async notifies */
+	pqsignal(SIGUSR2, SIG_IGN);
+	pqsignal(SIGFPE, FloatExceptionHandler);
+	pqsignal(SIGCHLD, SIG_DFL);
+
+	/* Early initialization */
+	BaseInit();
+
+	/*
+	 * Create a per-backend PGPROC struct in shared memory, except in the
+	 * EXEC_BACKEND case where this was done in SubPostmasterMain. We must do
+	 * this before we can use LWLocks (and in the EXEC_BACKEND case we already
+	 * had to do some stuff with LWLocks).
+	 */
+#ifndef EXEC_BACKEND
+	InitDummyProcess();
+#endif
+
+	/*
+	 * Create a memory context that we will do all our work in.  We do this so
+	 * that we can reset the context during error recovery and thereby avoid
+	 * possible memory leaks.
+	 */
+	avlauncher_cxt = AllocSetContextCreate(TopMemoryContext,
+										   "Autovacuum Launcher",
+										   ALLOCSET_DEFAULT_MINSIZE,
+										   ALLOCSET_DEFAULT_INITSIZE,
+										   ALLOCSET_DEFAULT_MAXSIZE);
+	MemoryContextSwitchTo(avlauncher_cxt);
+
+
+	/*
+	 * If an exception is encountered, processing resumes here.
+	 *
+	 * This code is heavily based on bgwriter.c, q.v.
+	 */
+	if (sigsetjmp(local_sigjmp_buf, 1) != 0)
+	{
+		/* since not using PG_TRY, must reset error stack by hand */
+		error_context_stack = NULL;
+
+		/* Prevents interrupts while cleaning up */
+		HOLD_INTERRUPTS();
+
+		/* Report the error to the server log */
+		EmitErrorReport();
+
+		/*
+		 * These operations are really just a minimal subset of
+		 * AbortTransaction().  We don't have very many resources to worry
+		 * about, but we do have LWLocks.
+		 */
+		LWLockReleaseAll();
+		AtEOXact_Files();
+
+		/*
+		 * Now return to normal top-level context and clear ErrorContext for
+		 * next time.
+		 */
+		MemoryContextSwitchTo(avlauncher_cxt);
+		FlushErrorState();
+
+		/* Flush any leaked data in the top-level context */
+		MemoryContextResetAndDeleteChildren(avlauncher_cxt);
+
+		/* Make sure pgstat also considers our stat data as gone */
+		pgstat_clear_snapshot();
+
+		/* Now we can allow interrupts again */
+		RESUME_INTERRUPTS();
+
+		/*
+		 * Sleep at least 1 second after any error.  We don't want to be
+		 * filling the error logs as fast as we can.
+		 */
+		pg_usleep(1000000L);
+	}
+
+	/* We can now handle ereport(ERROR) */
+	PG_exception_stack = &local_sigjmp_buf;
+
+	ereport(LOG,
+			(errmsg("autovacuum launcher started")));
+
+	PG_SETMASK(&UnBlockSig);
+
+	/*
+	 * take a nap before executing the first iteration, unless we were
+	 * requested an emergency run.
+	 */
+	if (autovacuum_start_daemon)
+		pg_usleep(autovacuum_naptime * 1000000L); 
+
+	for (;;)
+	{
+		TransactionId xidForceLimit;
+		ListCell *cell;
+		int		worker_pid;
+
+		/*
+		 * Emergency bailout if postmaster has died.  This is to avoid the
+		 * necessity for manual cleanup of all postmaster children.
+		 */
+		if (!PostmasterIsAlive(true))
+			exit(1);
+
+		if (avlauncher_shutdown_request)
+			break;
+
+		if (got_SIGHUP)
+		{
+			got_SIGHUP = false;
+			ProcessConfigFile(PGC_SIGHUP);
+		}
+
+		/*
+		 * if there's a worker already running, sleep until it
+		 * disappears.
+		 */
+		LWLockAcquire(AutovacuumLock, LW_SHARED);
+		worker_pid = AutoVacuumShmem->worker_pid;
+		LWLockRelease(AutovacuumLock);
+
+		if (worker_pid != 0)
+		{
+			PGPROC *proc = BackendPidGetProc(worker_pid);
+
+			if (proc != NULL && proc->isAutovacuum)
+				goto sleep;
+			else
+			{
+				/*
+				 * if the worker is not really running (or it's a process
+				 * that's not an autovacuum worker), remove the PID from shmem.
+				 * This should not happen, because either the worker exits
+				 * cleanly, in which case it'll remove the PID, or it dies, in
+				 * which case postmaster will cause a system reset cycle.
+				 */
+				LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
+				worker_pid = 0;
+				LWLockRelease(AutovacuumLock);
+			}
+		}
+
+		/* Get a list of databases */
+		dblist = autovac_get_database_list();
+
+		/*
+		 * Determine the oldest datfrozenxid/relfrozenxid that we will allow
+		 * to pass without forcing a vacuum.  (This limit can be tightened for
+		 * particular tables, but not loosened.)
+		 */
+		recentXid = ReadNewTransactionId();
+		xidForceLimit = recentXid - autovacuum_freeze_max_age;
+		/* ensure it's a "normal" XID, else TransactionIdPrecedes misbehaves */
+		if (xidForceLimit < FirstNormalTransactionId)
+			xidForceLimit -= FirstNormalTransactionId;
+
+		/*
+		 * Choose a database to connect to.  We pick the database that was least
+		 * recently auto-vacuumed, or one that needs vacuuming to prevent Xid
+		 * wraparound-related data loss.  If any db at risk of wraparound is
+		 * found, we pick the one with oldest datfrozenxid, independently of
+		 * autovacuum times.
+		 *
+		 * Note that a database with no stats entry is not considered, except for
+		 * Xid wraparound purposes.  The theory is that if no one has ever
+		 * connected to it since the stats were last initialized, it doesn't need
+		 * vacuuming.
+		 *
+		 * XXX This could be improved if we had more info about whether it needs
+		 * vacuuming before connecting to it.  Perhaps look through the pgstats
+		 * data for the database's tables?  One idea is to keep track of the
+		 * number of new and dead tuples per database in pgstats.  However it
+		 * isn't clear how to construct a metric that measures that and not cause
+		 * starvation for less busy databases.
+		 */
+		db = NULL;
+		for_xid_wrap = false;
+		foreach(cell, dblist)
+		{
+			autovac_dbase *tmp = lfirst(cell);
+
+			/* Find pgstat entry if any */
+			tmp->entry = pgstat_fetch_stat_dbentry(tmp->oid);
+
+			/* Check to see if this one is at risk of wraparound */
+			if (TransactionIdPrecedes(tmp->frozenxid, xidForceLimit))
+			{
+				if (db == NULL ||
+					TransactionIdPrecedes(tmp->frozenxid, db->frozenxid))
+					db = tmp;
+				for_xid_wrap = true;
+				continue;
+			}
+			else if (for_xid_wrap)
+				continue;			/* ignore not-at-risk DBs */
+
+			/*
+			 * Otherwise, skip a database with no pgstat entry; it means it
+			 * hasn't seen any activity.
+			 */
+			if (!tmp->entry)
+				continue;
+
+			/*
+			 * Remember the db with oldest autovac time.  (If we are here,
+			 * both tmp->entry and db->entry must be non-null.)
+			 */
+			if (db == NULL ||
+				tmp->entry->last_autovac_time < db->entry->last_autovac_time)
+				db = tmp;
+		}
+
+		/* Found a database -- process it */
+		if (db != NULL)
+		{
+			LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
+			AutoVacuumShmem->process_db = db->oid;
+			LWLockRelease(AutovacuumLock);
+
+			SendPostmasterSignal(PMSIGNAL_START_AUTOVAC_WORKER);
+		}
+		
+sleep:
+		/*
+		 * in emergency mode, exit immediately so that the postmaster can
+		 * request another run right away if needed.
+		 *
+		 * XXX -- maybe it would be better to handle this inside the launcher
+		 * itself.
+		 */
+		if (!autovacuum_start_daemon)
+			break;
+
+		/* have pgstat read the file again next time */
+		pgstat_clear_snapshot();
+
+		/* now sleep until the next autovac iteration */
+		pg_usleep(autovacuum_naptime * 1000000L); 
+	}
+
+	/* Normal exit from the autovac launcher is here */
+	ereport(LOG,
+			(errmsg("autovacuum launcher shutting down")));
+
+	proc_exit(0);		/* done */
+}
+
+/* SIGHUP: set flag to re-read config file at next convenient time */
+static void
+avl_sighup_handler(SIGNAL_ARGS)
+{
+	got_SIGHUP = true;
 }
 
+static void
+avlauncher_shutdown(SIGNAL_ARGS)
+{
+	avlauncher_shutdown_request = true;
+}
+
+/*
+ * avl_quickdie occurs when signalled SIGQUIT from postmaster.
+ *
+ * Some backend has bought the farm, so we need to stop what we're doing
+ * and exit.
+ */
+static void
+avl_quickdie(SIGNAL_ARGS)
+{
+	PG_SETMASK(&BlockSig);
+
+	/*
+	 * DO NOT proc_exit() -- we're here because shared memory may be
+	 * corrupted, so we don't want to try to clean up our transaction. Just
+	 * nail the windows shut and get out of town.
+	 *
+	 * Note we do exit(2) not exit(0).	This is to force the postmaster into a
+	 * system reset cycle if some idiot DBA sends a manual SIGQUIT to a random
+	 * backend.  This is necessary precisely because we don't clean up our
+	 * shared memory state.
+	 */
+	exit(2);
+}
+
+
+/********************************************************************
+ *                    AUTOVACUUM WORKER CODE
+ ********************************************************************/
+
 #ifdef EXEC_BACKEND
 /*
- * autovac_forkexec()
+ * forkexec routines for the autovacuum worker.
  *
- * Format up the arglist for the autovacuum process, then fork and exec.
+ * Format up the arglist, then fork and exec.
  */
 static pid_t
-autovac_forkexec(void)
+avworker_forkexec(void)
 {
 	char	   *av[10];
 	int			ac = 0;
 
 	av[ac++] = "postgres";
-	av[ac++] = "--forkautovac";
+	av[ac++] = "--forkavworker";
 	av[ac++] = NULL;			/* filled in by postmaster_forkexec */
 	av[ac] = NULL;
 
@@ -221,34 +579,71 @@ autovac_forkexec(void)
  * We need this set from the outside, before InitProcess is called
  */
 void
-AutovacuumIAm(void)
+AutovacuumWorkerIAm(void)
+{
+	am_autovacuum_worker = true;
+}
+#endif
+
+/*
+ * Main entry point for autovacuum worker process.
+ *
+ * This code is heavily based on pgarch.c, q.v.
+ */
+int
+StartAutoVacWorker(void)
 {
-	am_autovacuum = true;
+	pid_t		worker_pid;
+
+#ifdef EXEC_BACKEND
+	switch ((worker_pid = avworker_forkexec()))
+#else
+	switch ((worker_pid = fork_process()))
+#endif
+	{
+		case -1:
+			ereport(LOG,
+					(errmsg("could not fork autovacuum process: %m")));
+			return 0;
+
+#ifndef EXEC_BACKEND
+		case 0:
+			/* in postmaster child ... */
+			/* Close the postmaster's sockets */
+			ClosePostmasterPorts(false);
+
+			/* Lose the postmaster's on-exit routines */
+			on_exit_reset();
+
+			AutoVacWorkerMain(0, NULL);
+			break;
+#endif
+		default:
+			return (int) worker_pid;
+	}
+
+	/* shouldn't get here */
+	return 0;
 }
-#endif   /* EXEC_BACKEND */
 
 /*
- * AutoVacMain
+ * AutoVacWorkerMain
  */
 NON_EXEC_STATIC void
-AutoVacMain(int argc, char *argv[])
+AutoVacWorkerMain(int argc, char *argv[])
 {
-	ListCell   *cell;
-	List	   *dblist;
-	autovac_dbase *db;
-	TransactionId xidForceLimit;
-	bool		for_xid_wrap;
 	sigjmp_buf	local_sigjmp_buf;
+	Oid			dbid;
 
 	/* we are a postmaster subprocess now */
 	IsUnderPostmaster = true;
-	am_autovacuum = true;
+	am_autovacuum_worker = true;
 
 	/* reset MyProcPid */
 	MyProcPid = getpid();
 
 	/* Identify myself via ps */
-	init_ps_display("autovacuum process", "", "", "");
+	init_ps_display("autovacuum worker process", "", "", "");
 
 	SetProcessingMode(InitProcessing);
 
@@ -335,78 +730,24 @@ AutoVacMain(int argc, char *argv[])
 	 */
 	SetConfigOption("zero_damaged_pages", "false", PGC_SUSET, PGC_S_OVERRIDE);
 
-	/* Get a list of databases */
-	dblist = autovac_get_database_list();
-
 	/*
-	 * Determine the oldest datfrozenxid/relfrozenxid that we will allow
-	 * to pass without forcing a vacuum.  (This limit can be tightened for
-	 * particular tables, but not loosened.)
+	 * Get the database Id we're going to work on, and announce our PID
+	 * in the shared memory area.  We remove the database OID immediately
+	 * from the shared memory area.
 	 */
-	recentXid = ReadNewTransactionId();
-	xidForceLimit = recentXid - autovacuum_freeze_max_age;
-	/* ensure it's a "normal" XID, else TransactionIdPrecedes misbehaves */
-	if (xidForceLimit < FirstNormalTransactionId)
-		xidForceLimit -= FirstNormalTransactionId;
+	LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
 
-	/*
-	 * Choose a database to connect to.  We pick the database that was least
-	 * recently auto-vacuumed, or one that needs vacuuming to prevent Xid
-	 * wraparound-related data loss.  If any db at risk of wraparound is
-	 * found, we pick the one with oldest datfrozenxid,
-	 * independently of autovacuum times.
-	 *
-	 * Note that a database with no stats entry is not considered, except for
-	 * Xid wraparound purposes.  The theory is that if no one has ever
-	 * connected to it since the stats were last initialized, it doesn't need
-	 * vacuuming.
-	 *
-	 * XXX This could be improved if we had more info about whether it needs
-	 * vacuuming before connecting to it.  Perhaps look through the pgstats
-	 * data for the database's tables?  One idea is to keep track of the
-	 * number of new and dead tuples per database in pgstats.  However it
-	 * isn't clear how to construct a metric that measures that and not cause
-	 * starvation for less busy databases.
-	 */
-	db = NULL;
-	for_xid_wrap = false;
-	foreach(cell, dblist)
-	{
-		autovac_dbase *tmp = lfirst(cell);
+	dbid = AutoVacuumShmem->process_db;
+	AutoVacuumShmem->process_db = InvalidOid;
+	AutoVacuumShmem->worker_pid = MyProcPid;
 
-		/* Find pgstat entry if any */
-		tmp->entry = pgstat_fetch_stat_dbentry(tmp->oid);
+	LWLockRelease(AutovacuumLock);
 
-		/* Check to see if this one is at risk of wraparound */
-		if (TransactionIdPrecedes(tmp->frozenxid, xidForceLimit))
-		{
-			if (db == NULL ||
-				TransactionIdPrecedes(tmp->frozenxid, db->frozenxid))
-				db = tmp;
-			for_xid_wrap = true;
-			continue;
-		}
-		else if (for_xid_wrap)
-			continue;			/* ignore not-at-risk DBs */
-
-		/*
-		 * Otherwise, skip a database with no pgstat entry; it means it
-		 * hasn't seen any activity.
-		 */
-		if (!tmp->entry)
-			continue;
-
-		/*
-		 * Remember the db with oldest autovac time.  (If we are here,
-		 * both tmp->entry and db->entry must be non-null.)
-		 */
-		if (db == NULL ||
-			tmp->entry->last_autovac_time < db->entry->last_autovac_time)
-			db = tmp;
-	}
-
-	if (db)
+	if (OidIsValid(dbid))
 	{
+		char	*dbname;
+		PgStat_StatDBEntry *dbentry;
+
 		/*
 		 * Report autovac startup to the stats collector.  We deliberately do
 		 * this before InitPostgres, so that the last_autovac_time will get
@@ -415,7 +756,7 @@ AutoVacMain(int argc, char *argv[])
 		 * database, rather than making any progress on stuff it can connect
 		 * to.
 		 */
-		pgstat_report_autovac(db->oid);
+		pgstat_report_autovac(dbid);
 
 		/*
 		 * Connect to the selected database
@@ -423,11 +764,11 @@ AutoVacMain(int argc, char *argv[])
 		 * Note: if we have selected a just-deleted database (due to using
 		 * stale stats info), we'll fail and exit here.
 		 */
-		InitPostgres(db->name, NULL);
+		InitPostgres(NULL, dbid, NULL, &dbname);
 		SetProcessingMode(NormalProcessing);
-		set_ps_display(db->name, false);
+		set_ps_display(dbname, false);
 		ereport(DEBUG1,
-				(errmsg("autovacuum: processing database \"%s\"", db->name)));
+				(errmsg("autovacuum: processing database \"%s\"", dbname)));
 
 		/* Create the memory context where cross-transaction state is stored */
 		AutovacMemCxt = AllocSetContextCreate(TopMemoryContext,
@@ -436,13 +777,21 @@ AutoVacMain(int argc, char *argv[])
 											  ALLOCSET_DEFAULT_INITSIZE,
 											  ALLOCSET_DEFAULT_MAXSIZE);
 
-		/*
-		 * And do an appropriate amount of work
-		 */
-		do_autovacuum(db->entry);
+		/* And do an appropriate amount of work */
+		recentXid = ReadNewTransactionId();
+		dbentry = pgstat_fetch_stat_dbentry(dbid);
+		do_autovacuum(dbentry);
 	}
 
-	/* One iteration done, go away */
+	/*
+	 * Now remove our PID from shared memory, so that the launcher can start
+	 * another worker as soon as appropriate.
+	 */
+	LWLockAcquire(AutovacuumLock, LW_EXCLUSIVE);
+	AutoVacuumShmem->worker_pid = 0;
+	LWLockRelease(AutovacuumLock);
+
+	/* All done, go away */
 	proc_exit(0);
 }
 
@@ -450,7 +799,7 @@ AutoVacMain(int argc, char *argv[])
  * autovac_get_database_list
  *
  *		Return a list of all databases.  Note we cannot use pg_database,
- *		because we aren't connected yet; we use the flat database file.
+ *		because we aren't connected; we use the flat database file.
  */
 static List *
 autovac_get_database_list(void)
@@ -912,7 +1261,7 @@ autovacuum_do_vac_analyze(Oid relid, bool dovacuum, bool doanalyze,
 	vacstmt->analyze = doanalyze;
 	vacstmt->freeze_min_age = freeze_min_age;
 	vacstmt->verbose = false;
-	vacstmt->relation = NULL;	/* not used since we pass relids list */
+	vacstmt->relation = NULL;	/* not used since we pass a relids list */
 	vacstmt->va_cols = NIL;
 
 	/* Let pgstat know what we're doing */
@@ -1011,11 +1360,52 @@ autovac_init(void)
 }
 
 /*
- * IsAutoVacuumProcess
- *		Return whether this process is an autovacuum process.
+ * IsAutoVacuum functions
+ *		Return whether this is either a launcher autovacuum process or a worker
+ *		process.
  */
 bool
-IsAutoVacuumProcess(void)
+IsAutoVacuumLauncherProcess(void)
+{
+	return am_autovacuum_launcher;
+}
+
+bool
+IsAutoVacuumWorkerProcess(void)
 {
-	return am_autovacuum;
+	return am_autovacuum_worker;
+}
+
+
+/*
+ * AutoVacuumShmemSize
+ * 		Compute space needed for autovacuum-related shared memory
+ */
+Size
+AutoVacuumShmemSize(void)
+{
+	return sizeof(AutoVacuumShmemStruct);
+}
+
+/*
+ * AutoVacuumShmemInit
+ *		Allocate and initialize autovacuum-related shared memory
+ */
+void
+AutoVacuumShmemInit(void)
+{
+	bool        found;
+
+	AutoVacuumShmem = (AutoVacuumShmemStruct *)
+		ShmemInitStruct("AutoVacuum Data",
+						AutoVacuumShmemSize(),
+						&found);
+	if (AutoVacuumShmem == NULL)
+		ereport(FATAL,
+				(errcode(ERRCODE_OUT_OF_MEMORY),
+				 errmsg("not enough shared memory for autovacuum")));
+	if (found)
+		return;                 /* already initialized */
+
+	MemSet(AutoVacuumShmem, 0, sizeof(AutoVacuumShmemStruct));
 }
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 56ee4916d922cd042e3ba291df136604a6750924..d6e8cd220a0e720183559a32ce13e1417775941f 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -13,7 +13,7 @@
  *
  *	Copyright (c) 2001-2007, PostgreSQL Global Development Group
  *
- *	$PostgreSQL: pgsql/src/backend/postmaster/pgstat.c,v 1.146 2007/02/09 16:12:18 tgl Exp $
+ *	$PostgreSQL: pgsql/src/backend/postmaster/pgstat.c,v 1.147 2007/02/15 23:23:23 alvherre Exp $
  * ----------
  */
 #include "postgres.h"
@@ -930,7 +930,7 @@ pgstat_report_vacuum(Oid tableoid, bool shared,
 	msg.m_databaseid = shared ? InvalidOid : MyDatabaseId;
 	msg.m_tableoid = tableoid;
 	msg.m_analyze = analyze;
-	msg.m_autovacuum = IsAutoVacuumProcess();	/* is this autovacuum? */
+	msg.m_autovacuum = IsAutoVacuumWorkerProcess();	/* is this autovacuum? */
 	msg.m_vacuumtime = GetCurrentTimestamp();
 	msg.m_tuples = tuples;
 	pgstat_send(&msg, sizeof(msg));
@@ -955,7 +955,7 @@ pgstat_report_analyze(Oid tableoid, bool shared, PgStat_Counter livetuples,
 	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_ANALYZE);
 	msg.m_databaseid = shared ? InvalidOid : MyDatabaseId;
 	msg.m_tableoid = tableoid;
-	msg.m_autovacuum = IsAutoVacuumProcess();	/* is this autovacuum? */
+	msg.m_autovacuum = IsAutoVacuumWorkerProcess();	/* is this autovacuum? */
 	msg.m_analyzetime = GetCurrentTimestamp();
 	msg.m_live_tuples = livetuples;
 	msg.m_dead_tuples = deadtuples;
@@ -2280,8 +2280,8 @@ backend_read_statsfile(void)
 		return;
 	Assert(!pgStatRunningInCollector);
 
-	/* Autovacuum wants stats about all databases */
-	if (IsAutoVacuumProcess())
+	/* Autovacuum launcher wants stats about all databases */
+	if (IsAutoVacuumLauncherProcess())
 		pgStatDBHash = pgstat_read_statsfile(InvalidOid);
 	else
 		pgStatDBHash = pgstat_read_statsfile(MyDatabaseId);
@@ -2319,8 +2319,8 @@ pgstat_setup_memcxt(void)
 void
 pgstat_clear_snapshot(void)
 {
-	/* In an autovacuum process we keep the stats forever */
-	if (IsAutoVacuumProcess())
+	/* In an autovacuum worker process we keep the stats forever */
+	if (IsAutoVacuumWorkerProcess())
 		return;
 
 	/* Release memory, if any was allocated */
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index d0fe8451987a418f4332eb3f0db6b31723c55032..1917cd4185f79893afd90dcf83ed3cc5f849deac 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -37,7 +37,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/postmaster/postmaster.c,v 1.521 2007/02/13 19:18:54 tgl Exp $
+ *	  $PostgreSQL: pgsql/src/backend/postmaster/postmaster.c,v 1.522 2007/02/15 23:23:23 alvherre Exp $
  *
  * NOTES
  *
@@ -129,13 +129,14 @@
  * authorization phase).  This is used mainly to keep track of how many
  * children we have and send them appropriate signals when necessary.
  *
- * "Special" children such as the startup and bgwriter tasks are not in
- * this list.
+ * "Special" children such as the startup, bgwriter and autovacuum launcher
+ * tasks are not in this list.  Autovacuum worker processes are on it.
  */
 typedef struct bkend
 {
 	pid_t		pid;			/* process id of backend */
 	long		cancel_key;		/* cancel key for cancels for this backend */
+	bool		is_autovacuum;	/* is it an autovacuum process */
 } Backend;
 
 static Dllist *BackendList;
@@ -217,7 +218,8 @@ static bool FatalError = false; /* T if recovering from backend crash */
 bool		ClientAuthInProgress = false;		/* T during new-client
 												 * authentication */
 
-static bool force_autovac = false; /* received START_AUTOVAC signal */
+/* received START_AUTOVAC_LAUNCHER signal */
+static bool start_autovac_launcher = false;
 
 /*
  * State for assigning random salts and cancel keys.
@@ -270,9 +272,11 @@ static long PostmasterRandom(void);
 static void RandomSalt(char *cryptSalt, char *md5Salt);
 static void signal_child(pid_t pid, int signal);
 static void SignalChildren(int signal);
+static void SignalSomeChildren(int signal, bool only_autovac);
 static int	CountChildren(void);
 static bool CreateOptsFile(int argc, char *argv[], char *fullprogname);
 static pid_t StartChildProcess(int xlop);
+static void StartAutovacuumWorker(void);
 
 #ifdef EXEC_BACKEND
 
@@ -1145,13 +1149,12 @@ ServerLoop(void)
 		/*
 		 * Wait for something to happen.
 		 *
-		 * We wait at most one minute, or the minimum autovacuum delay, to
-		 * ensure that the other background tasks handled below get done even
-		 * when no requests are arriving.
+		 * We wait at most one minute, to ensure that the other background
+		 * tasks handled below get done even when no requests are arriving.
 		 */
 		memcpy((char *) &rmask, (char *) &readmask, sizeof(fd_set));
 
-		timeout.tv_sec = Min(60, autovacuum_naptime);
+		timeout.tv_sec = 60;
 		timeout.tv_usec = 0;
 
 		PG_SETMASK(&UnBlockSig);
@@ -1238,17 +1241,13 @@ ServerLoop(void)
 				signal_child(BgWriterPID, SIGUSR2);
 		}
 
-		/*
-		 * Start a new autovacuum process, if there isn't one running already.
-		 * (It'll die relatively quickly.)  We check that it's not started too
-		 * frequently in autovac_start.
-		 */
-		if ((AutoVacuumingActive() || force_autovac) && AutoVacPID == 0 &&
+		/* If we have lost the autovacuum launcher, try to start a new one */
+		if ((AutoVacuumingActive() || start_autovac_launcher) && AutoVacPID == 0 &&
 			StartupPID == 0 && !FatalError && Shutdown == NoShutdown)
 		{
-			AutoVacPID = autovac_start();
+			AutoVacPID = StartAutoVacLauncher();
 			if (AutoVacPID != 0)
-				force_autovac = false;	/* signal successfully processed */
+				start_autovac_launcher = false;	/* signal successfully processed */
 		}
 
 		/* If we have lost the archiver, try to start a new one */
@@ -1874,15 +1873,9 @@ pmdie(SIGNAL_ARGS)
 			ereport(LOG,
 					(errmsg("received smart shutdown request")));
 
-			/*
-			 * We won't wait out an autovacuum iteration ...
-			 */
-			if (AutoVacPID != 0)
-			{
-				/* Use statement cancel to shut it down */
-				signal_child(AutoVacPID, SIGINT);
-				break;			/* let reaper() handle this */
-			}
+			/* autovacuum workers are shut down immediately */
+			if (DLGetHead(BackendList))
+				SignalSomeChildren(SIGINT, true);
 
 			if (DLGetHead(BackendList))
 				break;			/* let reaper() handle this */
@@ -1904,6 +1897,9 @@ pmdie(SIGNAL_ARGS)
 			/* Tell pgstat to shut down too; nothing left for it to do */
 			if (PgStatPID != 0)
 				signal_child(PgStatPID, SIGQUIT);
+			/* Tell autovac launcher to shut down too */
+			if (AutoVacPID != 0)
+				signal_child(AutoVacPID, SIGTERM);
 			break;
 
 		case SIGINT:
@@ -1920,15 +1916,13 @@ pmdie(SIGNAL_ARGS)
 			ereport(LOG,
 					(errmsg("received fast shutdown request")));
 
-			if (DLGetHead(BackendList) || AutoVacPID != 0)
+			if (DLGetHead(BackendList))
 			{
 				if (!FatalError)
 				{
 					ereport(LOG,
 							(errmsg("aborting any active transactions")));
 					SignalChildren(SIGTERM);
-					if (AutoVacPID != 0)
-						signal_child(AutoVacPID, SIGTERM);
 					/* reaper() does the rest */
 				}
 				break;
@@ -1959,6 +1953,9 @@ pmdie(SIGNAL_ARGS)
 			/* Tell pgstat to shut down too; nothing left for it to do */
 			if (PgStatPID != 0)
 				signal_child(PgStatPID, SIGQUIT);
+			/* Tell autovac launcher to shut down too */
+			if (AutoVacPID != 0)
+				signal_child(AutoVacPID, SIGTERM);
 			break;
 
 		case SIGQUIT:
@@ -2073,8 +2070,8 @@ reaper(SIGNAL_ARGS)
 
 			/*
 			 * Go to shutdown mode if a shutdown request was pending.
-			 * Otherwise, try to start the archiver and stats collector too.
-			 * (We could, but don't, try to start autovacuum here.)
+			 * Otherwise, try to start the archiver, stats collector and
+			 * autovacuum launcher.
 			 */
 			if (Shutdown > NoShutdown && BgWriterPID != 0)
 				signal_child(BgWriterPID, SIGUSR2);
@@ -2084,6 +2081,9 @@ reaper(SIGNAL_ARGS)
 					PgArchPID = pgarch_start();
 				if (PgStatPID == 0)
 					PgStatPID = pgstat_start();
+				if (AutoVacuumingActive() && AutoVacPID == 0)
+					AutoVacPID = StartAutoVacLauncher();
+
 				/* at this point we are really open for business */
 				ereport(LOG,
 						(errmsg("database system is ready to accept connections")));
@@ -2141,18 +2141,16 @@ reaper(SIGNAL_ARGS)
 		}
 
 		/*
-		 * Was it the autovacuum process?  Normal or FATAL exit can be
-		 * ignored; we'll start a new one at the next iteration of the
-		 * postmaster's main loop, if necessary.  Any other exit condition
-		 * is treated as a crash.
+		 * Was it the autovacuum launcher?  Normal exit can be ignored; we'll
+		 * start a new one at the next iteration of the postmaster's main loop,
+		 * if necessary.  Any other exit condition is treated as a crash.
 		 */
 		if (AutoVacPID != 0 && pid == AutoVacPID)
 		{
 			AutoVacPID = 0;
-			autovac_stopped();
-			if (!EXIT_STATUS_0(exitstatus) && !EXIT_STATUS_1(exitstatus))
+			if (!EXIT_STATUS_0(exitstatus))
 				HandleChildCrash(pid, exitstatus,
-								 _("autovacuum process"));
+								 _("autovacuum launcher process"));
 			continue;
 		}
 
@@ -2230,7 +2228,7 @@ reaper(SIGNAL_ARGS)
 
 	if (Shutdown > NoShutdown)
 	{
-		if (DLGetHead(BackendList) || StartupPID != 0 || AutoVacPID != 0)
+		if (DLGetHead(BackendList) || StartupPID != 0)
 			goto reaper_done;
 		/* Start the bgwriter if not running */
 		if (BgWriterPID == 0)
@@ -2244,6 +2242,9 @@ reaper(SIGNAL_ARGS)
 		/* Tell pgstat to shut down too; nothing left for it to do */
 		if (PgStatPID != 0)
 			signal_child(PgStatPID, SIGQUIT);
+		/* Tell autovac launcher to shut down too */
+		if (AutoVacPID != 0)
+			signal_child(AutoVacPID, SIGTERM);
 	}
 
 reaper_done:
@@ -2371,7 +2372,7 @@ HandleChildCrash(int pid, int exitstatus, const char *procname)
 		signal_child(BgWriterPID, (SendStop ? SIGSTOP : SIGQUIT));
 	}
 
-	/* Take care of the autovacuum daemon too */
+	/* Take care of the autovacuum launcher too */
 	if (pid == AutoVacPID)
 		AutoVacPID = 0;
 	else if (AutoVacPID != 0 && !FatalError)
@@ -2501,10 +2502,22 @@ signal_child(pid_t pid, int signal)
 }
 
 /*
- * Send a signal to all backend children (but NOT special children)
+ * Send a signal to all backend children, including autovacuum workers (but NOT
+ * special children).
  */
 static void
 SignalChildren(int signal)
+{
+	SignalSomeChildren(signal, false);
+}
+
+/*
+ * Send a signal to all backend children, including autovacuum workers (but NOT
+ * special children).  If only_autovac is TRUE, only the autovacuum worker
+ * processes are signalled.
+ */
+static void
+SignalSomeChildren(int signal, bool only_autovac)
 {
 	Dlelem	   *curr;
 
@@ -2512,6 +2525,9 @@ SignalChildren(int signal)
 	{
 		Backend    *bp = (Backend *) DLE_VAL(curr);
 
+		if (only_autovac && !bp->is_autovacuum)
+			continue;
+
 		ereport(DEBUG4,
 				(errmsg_internal("sending signal %d to process %d",
 								 signal, (int) bp->pid)));
@@ -2523,6 +2539,8 @@ SignalChildren(int signal)
  * BackendStartup -- start backend process
  *
  * returns: STATUS_ERROR if the fork failed, STATUS_OK otherwise.
+ *
+ * Note: if you change this code, also consider StartAutovacuumWorker.
  */
 static int
 BackendStartup(Port *port)
@@ -3319,13 +3337,16 @@ SubPostmasterMain(int argc, char *argv[])
 	 * same address the postmaster used.
 	 */
 	if (strcmp(argv[1], "--forkbackend") == 0 ||
-		strcmp(argv[1], "--forkautovac") == 0 ||
+		strcmp(argv[1], "--forkavlauncher") == 0 ||
+		strcmp(argv[1], "--forkavworker") == 0 ||
 		strcmp(argv[1], "--forkboot") == 0)
 		PGSharedMemoryReAttach();
 
 	/* autovacuum needs this set before calling InitProcess */
-	if (strcmp(argv[1], "--forkautovac") == 0)
-		AutovacuumIAm();
+	if (strcmp(argv[1], "--forkavlauncher") == 0)
+		AutovacuumLauncherIAm();
+	if (strcmp(argv[1], "--forkavworker") == 0)
+		AutovacuumWorkerIAm();
 
 	/*
 	 * Start our win32 signal implementation. This has to be done after we
@@ -3420,7 +3441,24 @@ SubPostmasterMain(int argc, char *argv[])
 		BootstrapMain(argc - 2, argv + 2);
 		proc_exit(0);
 	}
-	if (strcmp(argv[1], "--forkautovac") == 0)
+	if (strcmp(argv[1], "--forkavlauncher") == 0)
+	{
+		/* Close the postmaster's sockets */
+		ClosePostmasterPorts(false);
+
+		/* Restore basic shared memory pointers */
+		InitShmemAccess(UsedShmemSegAddr);
+
+		/* Need a PGPROC to run CreateSharedMemoryAndSemaphores */
+		InitDummyProcess();
+
+		/* Attach process to shared data structures */
+		CreateSharedMemoryAndSemaphores(false, 0);
+
+		AutoVacLauncherMain(argc - 2, argv + 2);
+		proc_exit(0);
+	}
+	if (strcmp(argv[1], "--forkavworker") == 0)
 	{
 		/* Close the postmaster's sockets */
 		ClosePostmasterPorts(false);
@@ -3434,7 +3472,7 @@ SubPostmasterMain(int argc, char *argv[])
 		/* Attach process to shared data structures */
 		CreateSharedMemoryAndSemaphores(false, 0);
 
-		AutoVacMain(argc - 2, argv + 2);
+		AutoVacWorkerMain(argc - 2, argv + 2);
 		proc_exit(0);
 	}
 	if (strcmp(argv[1], "--forkarch") == 0)
@@ -3518,11 +3556,7 @@ sigusr1_handler(SIGNAL_ARGS)
 		 * See storage/ipc/sinval[adt].c for the use of this.
 		 */
 		if (Shutdown <= SmartShutdown)
-		{
 			SignalChildren(SIGUSR1);
-			if (AutoVacPID != 0)
-				signal_child(AutoVacPID, SIGUSR1);
-		}
 	}
 
 	if (CheckPostmasterSignal(PMSIGNAL_WAKEN_ARCHIVER) &&
@@ -3542,7 +3576,7 @@ sigusr1_handler(SIGNAL_ARGS)
 		signal_child(SysLoggerPID, SIGUSR1);
 	}
 
-	if (CheckPostmasterSignal(PMSIGNAL_START_AUTOVAC))
+	if (CheckPostmasterSignal(PMSIGNAL_START_AUTOVAC_LAUNCHER))
 	{
 		/*
 		 * Start one iteration of the autovacuum daemon, even if autovacuuming
@@ -3553,9 +3587,13 @@ sigusr1_handler(SIGNAL_ARGS)
 		 * that by launching another iteration as soon as the current one
 		 * completes.
 		 */
-		force_autovac = true;
+		start_autovac_launcher = true;
 	}
 
+	/* The autovacuum launcher wants us to start a worker process. */
+	if (CheckPostmasterSignal(PMSIGNAL_START_AUTOVAC_WORKER))
+		StartAutovacuumWorker();
+
 	PG_SETMASK(&UnBlockSig);
 
 	errno = save_errno;
@@ -3759,6 +3797,56 @@ StartChildProcess(int xlop)
 	return pid;
 }
 
+/*
+ * StartAutovacuumWorker
+ *		Start an autovac worker process.
+ *
+ * This function is here because it enters the resulting PID into the
+ * postmaster's private backends list.
+ *
+ * NB -- this code very roughly matches BackendStartup.
+ */
+static void
+StartAutovacuumWorker(void)
+{
+	Backend	   *bn;
+
+	/*
+	 * do nothing if not in condition to run a process.  This should not
+	 * actually happen, since the signal is only supposed to be sent by
+	 * autovacuum launcher when it's OK to do it, but test for it just in case.
+	 */
+	if (StartupPID != 0 || FatalError || Shutdown != NoShutdown)
+		return;
+
+	bn = (Backend *) malloc(sizeof(Backend));
+	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 */
+
+	if (bn->pid > 0)
+	{
+		DLAddHead(BackendList, DLNewElem(bn));
+#ifdef EXEC_BACKEND
+		ShmemBackendArrayAdd(bn);
+#endif
+	}
+	else
+	{
+		/* not much we can do */
+		ereport(LOG,
+				(errmsg("could not fork new process for autovacuum: %m")));
+		free(bn);
+	}
+}
 
 /*
  * Create the opts file
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index f8cccb5af7c4fe39d15e1f6bc6f085bd3749c3be..0296cbbcfc46f3b26651fc82030ceadf7398a609 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -8,7 +8,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/ipc/ipci.c,v 1.90 2007/01/05 22:19:37 momjian Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/ipc/ipci.c,v 1.91 2007/02/15 23:23:23 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -21,6 +21,7 @@
 #include "access/twophase.h"
 #include "miscadmin.h"
 #include "pgstat.h"
+#include "postmaster/autovacuum.h"
 #include "postmaster/bgwriter.h"
 #include "postmaster/postmaster.h"
 #include "storage/freespace.h"
@@ -109,6 +110,7 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
 		size = add_size(size, SInvalShmemSize());
 		size = add_size(size, FreeSpaceShmemSize());
 		size = add_size(size, BgWriterShmemSize());
+		size = add_size(size, AutoVacuumShmemSize());
 		size = add_size(size, BTreeShmemSize());
 #ifdef EXEC_BACKEND
 		size = add_size(size, ShmemBackendArraySize());
@@ -208,6 +210,7 @@ CreateSharedMemoryAndSemaphores(bool makePrivate, int port)
 	 */
 	PMSignalInit();
 	BgWriterShmemInit();
+	AutoVacuumShmemInit();
 
 	/*
 	 * Set up other modules that need some shared memory space
diff --git a/src/backend/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 6e2a0ce81c4297fd56fddd0ea88ec8be80941647..d32ae45c6befc8fd7ee8d2b3bca3e74f1525d5df 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -8,7 +8,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.183 2007/01/16 13:28:56 alvherre Exp $
+ *	  $PostgreSQL: pgsql/src/backend/storage/lmgr/proc.c,v 1.184 2007/02/15 23:23:23 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -259,7 +259,7 @@ InitProcess(void)
 	MyProc->databaseId = InvalidOid;
 	MyProc->roleId = InvalidOid;
 	MyProc->inVacuum = false;
-	MyProc->isAutovacuum = IsAutoVacuumProcess();
+	MyProc->isAutovacuum = IsAutoVacuumWorkerProcess();
 	MyProc->lwWaiting = false;
 	MyProc->lwExclusive = false;
 	MyProc->lwWaitLink = NULL;
@@ -392,7 +392,7 @@ InitDummyProcess(void)
 	MyProc->databaseId = InvalidOid;
 	MyProc->roleId = InvalidOid;
 	MyProc->inVacuum = false;
-	MyProc->isAutovacuum = false;
+	MyProc->isAutovacuum = IsAutoVacuumLauncherProcess(); /* is this needed? */
 	MyProc->lwWaiting = false;
 	MyProc->lwExclusive = false;
 	MyProc->lwWaitLink = NULL;
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 51eb8fabcf2fc82f4ddbdf4dd14cc3c8e06c237d..9ca8b981ee7bf22b00ce20c8d7c5107d33f3be49 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -8,7 +8,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/tcop/postgres.c,v 1.522 2007/02/10 14:58:55 petere Exp $
+ *	  $PostgreSQL: pgsql/src/backend/tcop/postgres.c,v 1.523 2007/02/15 23:23:23 alvherre Exp $
  *
  * NOTES
  *	  this is the "main" module of the postgres backend and
@@ -3139,7 +3139,7 @@ PostgresMain(int argc, char *argv[], const char *username)
 	 */
 	ereport(DEBUG3,
 			(errmsg_internal("InitPostgres")));
-	am_superuser = InitPostgres(dbname, username);
+	am_superuser = InitPostgres(dbname, InvalidOid, username, NULL);
 
 	SetProcessingMode(NormalProcessing);
 
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index a5967454afdb94a2a621a80aa59ee9dcf02d5ff3..32f7cc064152a211282ea7dc2a4c24fc09aa84d6 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -8,7 +8,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/utils/init/miscinit.c,v 1.161 2007/02/01 19:10:28 momjian Exp $
+ *	  $PostgreSQL: pgsql/src/backend/utils/init/miscinit.c,v 1.162 2007/02/15 23:23:23 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -401,7 +401,7 @@ InitializeSessionUserId(const char *rolename)
 	 *
 	 * We do not enforce them for the autovacuum process either.
 	 */
-	if (IsUnderPostmaster && !IsAutoVacuumProcess())
+	if (IsUnderPostmaster && !IsAutoVacuumWorkerProcess())
 	{
 		/*
 		 * Is role allowed to login at all?
@@ -462,7 +462,7 @@ void
 InitializeSessionUserIdStandalone(void)
 {
 	/* This function should only be called in a single-user backend. */
-	AssertState(!IsUnderPostmaster || IsAutoVacuumProcess());
+	AssertState(!IsUnderPostmaster || IsAutoVacuumWorkerProcess());
 
 	/* call only once */
 	AssertState(!OidIsValid(AuthenticatedUserId));
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index 8429af83bbebaafd1548d9d91ccb10b1af15ccac..8fdb3be75e2f1b68faad4c86d00f733ac60a9d89 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -8,7 +8,7 @@
  *
  *
  * IDENTIFICATION
- *	  $PostgreSQL: pgsql/src/backend/utils/init/postinit.c,v 1.173 2007/01/05 22:19:44 momjian Exp $
+ *	  $PostgreSQL: pgsql/src/backend/utils/init/postinit.c,v 1.174 2007/02/15 23:23:23 alvherre Exp $
  *
  *
  *-------------------------------------------------------------------------
@@ -47,6 +47,7 @@
 
 
 static bool FindMyDatabase(const char *name, Oid *db_id, Oid *db_tablespace);
+static bool FindMyDatabaseByOid(Oid dbid, char *dbname, Oid *db_tablespace);
 static void CheckMyDatabase(const char *name, bool am_superuser);
 static void InitCommunication(void);
 static void ShutdownPostgres(int code, Datum arg);
@@ -102,6 +103,48 @@ FindMyDatabase(const char *name, Oid *db_id, Oid *db_tablespace)
 	return result;
 }
 
+/*
+ * FindMyDatabaseByOid
+ *
+ * As above, but the actual database Id is known.  Return its name and the 
+ * tablespace OID.  Return TRUE if found, FALSE if not.  The same restrictions
+ * as FindMyDatabase apply.
+ */
+static bool
+FindMyDatabaseByOid(Oid dbid, char *dbname, Oid *db_tablespace)
+{
+	bool		result = false;
+	char	   *filename;
+	FILE	   *db_file;
+	Oid			db_id;
+	char		thisname[NAMEDATALEN];
+	TransactionId db_frozenxid;
+
+	filename = database_getflatfilename();
+	db_file = AllocateFile(filename, "r");
+	if (db_file == NULL)
+		ereport(FATAL,
+				(errcode_for_file_access(),
+				 errmsg("could not open file \"%s\": %m", filename)));
+
+	while (read_pg_database_line(db_file, thisname, &db_id,
+								 db_tablespace, &db_frozenxid))
+	{
+		if (dbid == db_id)
+		{
+			result = true;
+			strlcpy(dbname, thisname, NAMEDATALEN);
+			break;
+		}
+	}
+
+	FreeFile(db_file);
+	pfree(filename);
+
+	return result;
+}
+
+
 /*
  * CheckMyDatabase -- fetch information from the pg_database entry for our DB
  */
@@ -135,9 +178,9 @@ CheckMyDatabase(const char *name, bool am_superuser)
 	 * a way to recover from disabling all access to all databases, for
 	 * example "UPDATE pg_database SET datallowconn = false;".
 	 *
-	 * We do not enforce them for the autovacuum process either.
+	 * We do not enforce them for the autovacuum worker processes either.
 	 */
-	if (IsUnderPostmaster && !IsAutoVacuumProcess())
+	if (IsUnderPostmaster && !IsAutoVacuumWorkerProcess())
 	{
 		/*
 		 * Check that the database is currently allowing connections.
@@ -270,8 +313,11 @@ BaseInit(void)
  * InitPostgres
  *		Initialize POSTGRES.
  *
- * In bootstrap mode neither of the parameters are used.  In autovacuum
- * mode, the username parameter is not used.
+ * The database can be specified by name, using the in_dbname parameter, or by
+ * OID, using the dboid parameter.  In the latter case, the computed database
+ * name is passed out to the caller as a palloc'ed string in out_dbname.
+ *
+ * In bootstrap mode no parameters are used.
  *
  * The return value indicates whether the userID is a superuser.  (That
  * can only be tested inside a transaction, so we want to do it during
@@ -285,12 +331,14 @@ BaseInit(void)
  * --------------------------------
  */
 bool
-InitPostgres(const char *dbname, const char *username)
+InitPostgres(const char *in_dbname, Oid dboid, const char *username,
+			 char **out_dbname)
 {
 	bool		bootstrap = IsBootstrapProcessingMode();
-	bool		autovacuum = IsAutoVacuumProcess();
+	bool		autovacuum = IsAutoVacuumWorkerProcess();
 	bool		am_superuser;
 	char	   *fullpath;
+	char		dbname[NAMEDATALEN];
 
 	/*
 	 * Set up the global variables holding database id and path.  But note we
@@ -307,15 +355,32 @@ InitPostgres(const char *dbname, const char *username)
 	else
 	{
 		/*
-		 * Find oid and tablespace of the database we're about to open. Since
-		 * we're not yet up and running we have to use the hackish
-		 * FindMyDatabase, which looks in the flat-file copy of pg_database.
+		 * Find tablespace of the database we're about to open. Since we're not
+		 * yet up and running we have to use one of the hackish FindMyDatabase
+		 * variants, which look in the flat-file copy of pg_database.
+		 *
+		 * If the in_dbname param is NULL, lookup database by OID.
 		 */
-		if (!FindMyDatabase(dbname, &MyDatabaseId, &MyDatabaseTableSpace))
-			ereport(FATAL,
-					(errcode(ERRCODE_UNDEFINED_DATABASE),
-					 errmsg("database \"%s\" does not exist",
-							dbname)));
+		if (in_dbname == NULL)
+		{
+			if (!FindMyDatabaseByOid(dboid, dbname, &MyDatabaseTableSpace))
+				ereport(FATAL,
+						(errcode(ERRCODE_UNDEFINED_DATABASE),
+						 errmsg("database %u does not exist", dboid)));
+			MyDatabaseId = dboid;
+			/* pass the database name to the caller */
+			*out_dbname = pstrdup(dbname);
+		}
+		else
+		{
+			if (!FindMyDatabase(in_dbname, &MyDatabaseId, &MyDatabaseTableSpace))
+				ereport(FATAL,
+						(errcode(ERRCODE_UNDEFINED_DATABASE),
+						 errmsg("database \"%s\" does not exist",
+								in_dbname)));
+			/* our database name is gotten from the caller */
+			strlcpy(dbname, in_dbname, NAMEDATALEN);
+		}
 	}
 
 	fullpath = GetDatabasePath(MyDatabaseId, MyDatabaseTableSpace);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 88595112f57f005f69978f1a832d1c2389eb23e0..29a450cab60e087c75d5668f6e58563021de2984 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -13,7 +13,7 @@
  * Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
- * $PostgreSQL: pgsql/src/include/miscadmin.h,v 1.191 2007/01/05 22:19:50 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/miscadmin.h,v 1.192 2007/02/15 23:23:23 alvherre Exp $
  *
  * NOTES
  *	  some of the information in this file should be moved to other files.
@@ -302,7 +302,8 @@ extern ProcessingMode Mode;
  *****************************************************************************/
 
 /* in utils/init/postinit.c */
-extern bool InitPostgres(const char *dbname, const char *username);
+extern bool InitPostgres(const char *in_dbname, Oid dboid, const char *username,
+			 char **out_dbname);
 extern void BaseInit(void);
 
 /* in utils/init/miscinit.c */
diff --git a/src/include/postmaster/autovacuum.h b/src/include/postmaster/autovacuum.h
index 7b3ad7e55098d5da33381021ec2c8c34aafec8d5..facf9de52b2ca24b8415f79ae5ea395710cf5adc 100644
--- a/src/include/postmaster/autovacuum.h
+++ b/src/include/postmaster/autovacuum.h
@@ -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.7 2007/01/16 13:28:57 alvherre Exp $
+ * $PostgreSQL: pgsql/src/include/postmaster/autovacuum.h,v 1.8 2007/02/15 23:23:23 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -27,16 +27,23 @@ extern int	autovacuum_vac_cost_limit;
 
 /* Status inquiry functions */
 extern bool AutoVacuumingActive(void);
-extern bool IsAutoVacuumProcess(void);
+extern bool IsAutoVacuumLauncherProcess(void);
+extern bool IsAutoVacuumWorkerProcess(void);
 
 /* Functions to start autovacuum process, called from postmaster */
 extern void autovac_init(void);
-extern int	autovac_start(void);
-extern void autovac_stopped(void);
+extern int	StartAutoVacLauncher(void);
+extern int	StartAutoVacWorker(void);
 
 #ifdef EXEC_BACKEND
-extern void AutoVacMain(int argc, char *argv[]);
-extern void AutovacuumIAm(void);
+extern void AutoVacLauncherMain(int argc, char *argv[]);
+extern void AutoVacWorkerMain(int argc, char *argv[]);
+extern void AutovacuumWorkerIAm(void);
+extern void AutovacuumLauncherIAm(void);
 #endif
 
+/* shared memory stuff */
+extern Size AutoVacuumShmemSize(void);
+extern void AutoVacuumShmemInit(void);
+
 #endif   /* AUTOVACUUM_H */
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index cff28815d9def8c9b66667a9929c34619151f88e..b4503f921352ad615fb8d4fc3cf9d98133b35dec 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -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/storage/lwlock.h,v 1.33 2007/01/05 22:19:58 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/storage/lwlock.h,v 1.34 2007/02/15 23:23:23 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -61,6 +61,8 @@ typedef enum LWLockId
 	TablespaceCreateLock,
 	BtreeVacuumLock,
 	AddinShmemInitLock,
+	AutovacuumLock,
+	/* Individual lock IDs end here */
 	FirstBufMappingLock,
 	FirstLockMgrLock = FirstBufMappingLock + NUM_BUFFER_PARTITIONS,
 
diff --git a/src/include/storage/pmsignal.h b/src/include/storage/pmsignal.h
index bc705fc13626c4ad1eaec4565b7f7f2eeef28ab4..8142ee0f2c462fd34412a7aa7b8685d674b0a946 100644
--- a/src/include/storage/pmsignal.h
+++ b/src/include/storage/pmsignal.h
@@ -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/storage/pmsignal.h,v 1.16 2007/01/05 22:19:58 momjian Exp $
+ * $PostgreSQL: pgsql/src/include/storage/pmsignal.h,v 1.17 2007/02/15 23:23:23 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -26,7 +26,8 @@ typedef enum
 	PMSIGNAL_WAKEN_CHILDREN,	/* send a SIGUSR1 signal to all backends */
 	PMSIGNAL_WAKEN_ARCHIVER,	/* send a NOTIFY signal to xlog archiver */
 	PMSIGNAL_ROTATE_LOGFILE,	/* send SIGUSR1 to syslogger to rotate logfile */
-	PMSIGNAL_START_AUTOVAC,		/* start an autovacuum iteration */
+	PMSIGNAL_START_AUTOVAC_LAUNCHER, /* start an autovacuum launcher */
+	PMSIGNAL_START_AUTOVAC_WORKER, /* start an autovacuum worker */
 
 	NUM_PMSIGNALS				/* Must be last value of enum! */
 } PMSignalReason;
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index b86f210fbe850fdf0e8a94d2d481e61fc60d5737..3e296fb0a3a8ffe3e41ca6dbd5542419c2047369 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -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/storage/proc.h,v 1.93 2007/01/16 13:28:57 alvherre Exp $
+ * $PostgreSQL: pgsql/src/include/storage/proc.h,v 1.94 2007/02/15 23:23:23 alvherre Exp $
  *
  *-------------------------------------------------------------------------
  */
@@ -121,7 +121,7 @@ typedef struct PROC_HDR
  * We set aside some extra PGPROC structures for "dummy" processes,
  * ie things that aren't full-fledged backends but need shmem access.
  */
-#define NUM_DUMMY_PROCS		2
+#define NUM_DUMMY_PROCS		3
 
 
 /* configurable options */