citus/src/backend/distributed/utils/maintenanced.c

795 lines
21 KiB
C

/*-------------------------------------------------------------------------
*
* maintenanced.c
* Background worker run for each citus using database in a postgres
* cluster.
*
* This file provides infrastructure for launching exactly one a background
* worker for every database in which citus is used. That background worker
* can then perform work like deadlock detection, prepared transaction
* recovery, and cleanup.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include <time.h>
#include "miscadmin.h"
#include "pgstat.h"
#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_extension.h"
#include "citus_version.h"
#include "catalog/pg_namespace.h"
#include "commands/async.h"
#include "commands/extension.h"
#include "libpq/pqsignal.h"
#include "catalog/namespace.h"
#include "distributed/citus_safe_lib.h"
#include "distributed/distributed_deadlock_detection.h"
#include "distributed/maintenanced.h"
#include "distributed/master_protocol.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/statistics_collection.h"
#include "distributed/transaction_recovery.h"
#include "distributed/version_compat.h"
#include "nodes/makefuncs.h"
#include "postmaster/bgworker.h"
#include "postmaster/postmaster.h"
#include "nodes/makefuncs.h"
#include "storage/ipc.h"
#include "storage/proc.h"
#include "storage/latch.h"
#include "storage/lmgr.h"
#include "storage/lwlock.h"
#include "tcop/tcopprot.h"
#include "utils/memutils.h"
#include "utils/lsyscache.h"
/*
* Shared memory data for all maintenance workers.
*/
typedef struct MaintenanceDaemonControlData
{
/*
* Lock protecting the shared memory state. This is to be taken when
* looking up (shared mode) or inserting (exclusive mode) per-database
* data in MaintenanceDaemonDBHash.
*/
int trancheId;
char *lockTrancheName;
LWLock lock;
} MaintenanceDaemonControlData;
/*
* Per database worker state.
*/
typedef struct MaintenanceDaemonDBData
{
/* hash key: database to run on */
Oid databaseOid;
/* information: which user to use */
Oid userOid;
pid_t workerPid;
bool daemonStarted;
bool triggerMetadataSync;
Latch *latch; /* pointer to the background worker's latch */
} MaintenanceDaemonDBData;
/* config variable for distributed deadlock detection timeout */
double DistributedDeadlockDetectionTimeoutFactor = 2.0;
int Recover2PCInterval = 60000;
/* config variables for metadata sync timeout */
int MetadataSyncInterval = 60000;
int MetadataSyncRetryInterval = 5000;
static shmem_startup_hook_type prev_shmem_startup_hook = NULL;
static MaintenanceDaemonControlData *MaintenanceDaemonControl = NULL;
/*
* Hash-table of workers, one entry for each database with citus
* activated.
*/
static HTAB *MaintenanceDaemonDBHash;
static volatile sig_atomic_t got_SIGHUP = false;
static void MaintenanceDaemonSigTermHandler(SIGNAL_ARGS);
static void MaintenanceDaemonSigHupHandler(SIGNAL_ARGS);
static size_t MaintenanceDaemonShmemSize(void);
static void MaintenanceDaemonShmemInit(void);
static void MaintenanceDaemonErrorContext(void *arg);
static bool LockCitusExtension(void);
static bool MetadataSyncTriggeredCheckAndReset(MaintenanceDaemonDBData *dbData);
/*
* InitializeMaintenanceDaemon, called at server start, is responsible for
* requesting shared memory and related infrastructure required by maintenance
* daemons.
*/
void
InitializeMaintenanceDaemon(void)
{
if (!IsUnderPostmaster)
{
RequestAddinShmemSpace(MaintenanceDaemonShmemSize());
}
prev_shmem_startup_hook = shmem_startup_hook;
shmem_startup_hook = MaintenanceDaemonShmemInit;
}
/*
* InitializeMaintenanceDaemonBackend, called at backend start and
* configuration changes, is responsible for starting a per-database
* maintenance worker if necessary.
*/
void
InitializeMaintenanceDaemonBackend(void)
{
Oid extensionOwner = CitusExtensionOwner();
bool found;
LWLockAcquire(&MaintenanceDaemonControl->lock, LW_EXCLUSIVE);
MaintenanceDaemonDBData *dbData = (MaintenanceDaemonDBData *) hash_search(
MaintenanceDaemonDBHash,
&
MyDatabaseId,
HASH_ENTER_NULL,
&found);
if (dbData == NULL)
{
/* FIXME: better message, reference relevant guc in hint */
ereport(ERROR, (errmsg("ran out of database slots")));
}
/* maintenance daemon can ignore itself */
if (dbData->workerPid == MyProcPid)
{
LWLockRelease(&MaintenanceDaemonControl->lock);
return;
}
if (!found || !dbData->daemonStarted)
{
BackgroundWorker worker;
BackgroundWorkerHandle *handle = NULL;
dbData->userOid = extensionOwner;
memset(&worker, 0, sizeof(worker));
SafeSnprintf(worker.bgw_name, sizeof(worker.bgw_name),
"Citus Maintenance Daemon: %u/%u",
MyDatabaseId, extensionOwner);
/* request ability to connect to target database */
worker.bgw_flags = BGWORKER_SHMEM_ACCESS | BGWORKER_BACKEND_DATABASE_CONNECTION;
/*
* No point in getting started before able to run query, but we do
* want to get started on Hot-Standby.
*/
worker.bgw_start_time = BgWorkerStart_ConsistentState;
/* Restart after a bit after errors, but don't bog the system. */
worker.bgw_restart_time = 5;
strcpy_s(worker.bgw_library_name,
sizeof(worker.bgw_library_name), "citus");
strcpy_s(worker.bgw_function_name, sizeof(worker.bgw_library_name),
"CitusMaintenanceDaemonMain");
worker.bgw_main_arg = ObjectIdGetDatum(MyDatabaseId);
memcpy_s(worker.bgw_extra, sizeof(worker.bgw_extra), &extensionOwner,
sizeof(Oid));
worker.bgw_notify_pid = MyProcPid;
if (!RegisterDynamicBackgroundWorker(&worker, &handle))
{
ereport(ERROR, (errmsg("could not start maintenance background worker"),
errhint("Increasing max_worker_processes might help.")));
}
dbData->daemonStarted = true;
dbData->workerPid = 0;
dbData->triggerMetadataSync = false;
LWLockRelease(&MaintenanceDaemonControl->lock);
pid_t pid;
WaitForBackgroundWorkerStartup(handle, &pid);
pfree(handle);
}
else
{
Assert(dbData->daemonStarted);
/*
* If owner of extension changed, wake up daemon. It'll notice and
* restart.
*/
if (dbData->userOid != extensionOwner)
{
dbData->userOid = extensionOwner;
if (dbData->latch)
{
SetLatch(dbData->latch);
}
}
LWLockRelease(&MaintenanceDaemonControl->lock);
}
}
/*
* CitusMaintenanceDaemonMain is the maintenance daemon's main routine, it'll
* be started by the background worker infrastructure. If it errors out,
* it'll be restarted after a few seconds.
*/
void
CitusMaintenanceDaemonMain(Datum main_arg)
{
Oid databaseOid = DatumGetObjectId(main_arg);
TimestampTz nextStatsCollectionTime USED_WITH_LIBCURL_ONLY =
TimestampTzPlusMilliseconds(GetCurrentTimestamp(), 60 * 1000);
bool retryStatsCollection USED_WITH_LIBCURL_ONLY = false;
ErrorContextCallback errorCallback;
TimestampTz lastRecoveryTime = 0;
TimestampTz nextMetadataSyncTime = 0;
/*
* Look up this worker's configuration.
*/
LWLockAcquire(&MaintenanceDaemonControl->lock, LW_SHARED);
MaintenanceDaemonDBData *myDbData = (MaintenanceDaemonDBData *)
hash_search(MaintenanceDaemonDBHash, &databaseOid,
HASH_FIND, NULL);
if (!myDbData || myDbData->workerPid != 0)
{
/*
* When the database crashes, background workers are restarted, but
* the state in shared memory is lost. In that case, we exit and
* wait for a session to call InitializeMaintenanceDaemonBackend
* to properly add it to the hash.
* Alternatively, don't continue if another worker exists.
*/
proc_exit(0);
}
/* from this point, DROP DATABASE will attempt to kill the worker */
myDbData->workerPid = MyProcPid;
/* wire up signals */
pqsignal(SIGTERM, MaintenanceDaemonSigTermHandler);
pqsignal(SIGHUP, MaintenanceDaemonSigHupHandler);
BackgroundWorkerUnblockSignals();
myDbData->latch = MyLatch;
LWLockRelease(&MaintenanceDaemonControl->lock);
/*
* Setup error context so log messages can be properly attributed. Some of
* them otherwise sound like they might be from a normal user connection.
* Do so before setting up signals etc, so we never exit without the
* context setup.
*/
memset(&errorCallback, 0, sizeof(errorCallback));
errorCallback.callback = MaintenanceDaemonErrorContext;
errorCallback.arg = (void *) myDbData;
errorCallback.previous = error_context_stack;
error_context_stack = &errorCallback;
elog(LOG, "starting maintenance daemon on database %u user %u",
databaseOid, myDbData->userOid);
/* connect to database, after that we can actually access catalogs */
BackgroundWorkerInitializeConnectionByOid(databaseOid, myDbData->userOid, 0);
/* make worker recognizable in pg_stat_activity */
pgstat_report_appname("Citus Maintenance Daemon");
/* enter main loop */
for (;;)
{
int rc;
int latchFlags = WL_LATCH_SET | WL_TIMEOUT | WL_POSTMASTER_DEATH;
double timeout = 10000.0; /* use this if the deadlock detection is disabled */
bool foundDeadlock = false;
CHECK_FOR_INTERRUPTS();
Assert(myDbData->workerPid == MyProcPid);
/*
* XXX: Each task should clear the metadata cache before every iteration
* by calling InvalidateMetadataSystemCache(), because otherwise it
* might contain stale OIDs. It appears that in some cases invalidation
* messages for a DROP EXTENSION may arrive during these tasks and
* this causes us to cache a stale pg_dist_node OID. We'd actually expect
* all invalidations to arrive after obtaining a lock in LockCitusExtension.
*/
/*
* Perform Work. If a specific task needs to be called sooner than
* timeout indicates, it's ok to lower it to that value. Expensive
* tasks should do their own time math about whether to re-run checks.
*/
#ifdef HAVE_LIBCURL
if (EnableStatisticsCollection &&
GetCurrentTimestamp() >= nextStatsCollectionTime)
{
bool statsCollectionSuccess = false;
InvalidateMetadataSystemCache();
StartTransactionCommand();
/*
* Lock the extension such that it cannot be dropped or created
* concurrently. Skip statistics collection if citus extension is
* not accessible.
*
* Similarly, we skip statistics collection if there exists any
* version mismatch or the extension is not fully created yet.
*/
if (!LockCitusExtension())
{
ereport(DEBUG1, (errmsg("could not lock the citus extension, "
"skipping statistics collection")));
}
else if (CheckCitusVersion(DEBUG1) && CitusHasBeenLoaded())
{
FlushDistTableCache();
WarnIfSyncDNS();
statsCollectionSuccess = CollectBasicUsageStatistics();
}
/*
* If statistics collection was successful the next collection is
* 24-hours later. Also, if this was a retry attempt we don't do
* any more retries until 24-hours later, so we limit number of
* retries to one.
*/
if (statsCollectionSuccess || retryStatsCollection)
{
nextStatsCollectionTime =
TimestampTzPlusMilliseconds(GetCurrentTimestamp(),
STATS_COLLECTION_TIMEOUT_MILLIS);
retryStatsCollection = false;
}
else
{
nextStatsCollectionTime =
TimestampTzPlusMilliseconds(GetCurrentTimestamp(),
STATS_COLLECTION_RETRY_TIMEOUT_MILLIS);
retryStatsCollection = true;
}
CommitTransactionCommand();
}
#endif
if (!RecoveryInProgress() &&
(MetadataSyncTriggeredCheckAndReset(myDbData) ||
GetCurrentTimestamp() >= nextMetadataSyncTime))
{
bool metadataSyncFailed = false;
InvalidateMetadataSystemCache();
StartTransactionCommand();
/*
* Some functions in ruleutils.c, which we use to get the DDL for
* metadata propagation, require an active snapshot.
*/
PushActiveSnapshot(GetTransactionSnapshot());
if (!LockCitusExtension())
{
ereport(DEBUG1, (errmsg("could not lock the citus extension, "
"skipping metadata sync")));
}
else if (CheckCitusVersion(DEBUG1) && CitusHasBeenLoaded())
{
MetadataSyncResult result = SyncMetadataToNodes();
metadataSyncFailed = (result != METADATA_SYNC_SUCCESS);
/*
* Notification means we had an attempt on synchronization
* without being blocked for pg_dist_node access.
*/
if (result != METADATA_SYNC_FAILED_LOCK)
{
Async_Notify(METADATA_SYNC_CHANNEL, NULL);
}
}
PopActiveSnapshot();
CommitTransactionCommand();
ProcessCompletedNotifies();
int64 nextTimeout = metadataSyncFailed ? MetadataSyncRetryInterval :
MetadataSyncInterval;
nextMetadataSyncTime =
TimestampTzPlusMilliseconds(GetCurrentTimestamp(), nextTimeout);
timeout = Min(timeout, nextTimeout);
}
/*
* If enabled, run 2PC recovery on primary nodes (where !RecoveryInProgress()),
* since we'll write to the pg_dist_transaction log.
*/
if (Recover2PCInterval > 0 && !RecoveryInProgress() &&
TimestampDifferenceExceeds(lastRecoveryTime, GetCurrentTimestamp(),
Recover2PCInterval))
{
int recoveredTransactionCount = 0;
InvalidateMetadataSystemCache();
StartTransactionCommand();
if (!LockCitusExtension())
{
ereport(DEBUG1, (errmsg("could not lock the citus extension, "
"skipping 2PC recovery")));
}
else if (CheckCitusVersion(DEBUG1) && CitusHasBeenLoaded())
{
/*
* Record last recovery time at start to ensure we run once per
* Recover2PCInterval even if RecoverTwoPhaseCommits takes some time.
*/
lastRecoveryTime = GetCurrentTimestamp();
recoveredTransactionCount = RecoverTwoPhaseCommits();
}
CommitTransactionCommand();
if (recoveredTransactionCount > 0)
{
ereport(LOG, (errmsg("maintenance daemon recovered %d distributed "
"transactions",
recoveredTransactionCount)));
}
/* make sure we don't wait too long */
timeout = Min(timeout, Recover2PCInterval);
}
/* the config value -1 disables the distributed deadlock detection */
if (DistributedDeadlockDetectionTimeoutFactor != -1.0)
{
double deadlockTimeout =
DistributedDeadlockDetectionTimeoutFactor * (double) DeadlockTimeout;
InvalidateMetadataSystemCache();
StartTransactionCommand();
/*
* We skip the deadlock detection if citus extension
* is not accessible.
*
* Similarly, we skip to run the deadlock checks if
* there exists any version mismatch or the extension
* is not fully created yet.
*/
if (!LockCitusExtension())
{
ereport(DEBUG1, (errmsg("could not lock the citus extension, "
"skipping deadlock detection")));
}
else if (CheckCitusVersion(DEBUG1) && CitusHasBeenLoaded())
{
foundDeadlock = CheckForDistributedDeadlocks();
}
CommitTransactionCommand();
/*
* If we find any deadlocks, run the distributed deadlock detection
* more often since it is quite possible that there are other
* deadlocks need to be resolved.
*
* Thus, we use 1/20 of the calculated value. With the default
* values (i.e., deadlock_timeout 1 seconds,
* citus.distributed_deadlock_detection_factor 2), we'd be able to cancel
* ~10 distributed deadlocks per second.
*/
if (foundDeadlock)
{
deadlockTimeout = deadlockTimeout / 20.0;
}
/* make sure we don't wait too long */
timeout = Min(timeout, deadlockTimeout);
}
/*
* Wait until timeout, or until somebody wakes us up. Also cast the timeout to
* integer where we've calculated it using double for not losing the precision.
*/
rc = WaitLatch(MyLatch, latchFlags, (long) timeout, PG_WAIT_EXTENSION);
/* emergency bailout if postmaster has died */
if (rc & WL_POSTMASTER_DEATH)
{
proc_exit(1);
}
if (rc & WL_LATCH_SET)
{
ResetLatch(MyLatch);
CHECK_FOR_INTERRUPTS();
/* check for changed configuration */
if (myDbData->userOid != GetSessionUserId())
{
/*
* Reset myDbData->daemonStarted so InitializeMaintenanceDaemonBackend()
* notices this is a restart.
*/
LWLockAcquire(&MaintenanceDaemonControl->lock, LW_EXCLUSIVE);
myDbData->daemonStarted = false;
myDbData->workerPid = 0;
LWLockRelease(&MaintenanceDaemonControl->lock);
/* return code of 1 requests worker restart */
proc_exit(1);
}
/*
* Could also add code checking whether extension still exists,
* but that'd complicate things a bit, because we'd have to delete
* the shared memory entry. There'd potentially be a race
* condition where the extension gets re-created, checking that
* this entry still exists, and it getting deleted just after.
* Doesn't seem worth catering for that.
*/
}
if (got_SIGHUP)
{
got_SIGHUP = false;
ProcessConfigFile(PGC_SIGHUP);
}
}
}
/*
* MaintenanceDaemonShmemSize computes how much shared memory is required.
*/
static size_t
MaintenanceDaemonShmemSize(void)
{
Size size = 0;
size = add_size(size, sizeof(MaintenanceDaemonControlData));
/*
* We request enough shared memory to have one hash-table entry for each
* worker process. We couldn't start more anyway, so there's little point
* in allocating more.
*/
Size hashSize = hash_estimate_size(max_worker_processes,
sizeof(MaintenanceDaemonDBData));
size = add_size(size, hashSize);
return size;
}
/*
* MaintenanceDaemonShmemInit initializes the requested shared memory for the
* maintenance daemon.
*/
static void
MaintenanceDaemonShmemInit(void)
{
bool alreadyInitialized = false;
HASHCTL hashInfo;
LWLockAcquire(AddinShmemInitLock, LW_EXCLUSIVE);
MaintenanceDaemonControl =
(MaintenanceDaemonControlData *) ShmemInitStruct("Citus Maintenance Daemon",
MaintenanceDaemonShmemSize(),
&alreadyInitialized);
/*
* Might already be initialized on EXEC_BACKEND type platforms that call
* shared library initialization functions in every backend.
*/
if (!alreadyInitialized)
{
MaintenanceDaemonControl->trancheId = LWLockNewTrancheId();
MaintenanceDaemonControl->lockTrancheName = "Citus Maintenance Daemon";
LWLockRegisterTranche(MaintenanceDaemonControl->trancheId,
MaintenanceDaemonControl->lockTrancheName);
LWLockInitialize(&MaintenanceDaemonControl->lock,
MaintenanceDaemonControl->trancheId);
}
memset(&hashInfo, 0, sizeof(hashInfo));
hashInfo.keysize = sizeof(Oid);
hashInfo.entrysize = sizeof(MaintenanceDaemonDBData);
hashInfo.hash = tag_hash;
int hashFlags = (HASH_ELEM | HASH_FUNCTION);
MaintenanceDaemonDBHash = ShmemInitHash("Maintenance Database Hash",
max_worker_processes, max_worker_processes,
&hashInfo, hashFlags);
LWLockRelease(AddinShmemInitLock);
if (prev_shmem_startup_hook != NULL)
{
prev_shmem_startup_hook();
}
}
/* MaintenanceDaemonSigTermHandler calls proc_exit(0) */
static void
MaintenanceDaemonSigTermHandler(SIGNAL_ARGS)
{
proc_exit(0);
}
/*
* MaintenanceDaemonSigHupHandler set a flag to re-read config file at next
* convenient time.
*/
static void
MaintenanceDaemonSigHupHandler(SIGNAL_ARGS)
{
int save_errno = errno;
got_SIGHUP = true;
if (MyProc != NULL)
{
SetLatch(&MyProc->procLatch);
}
errno = save_errno;
}
/*
* MaintenanceDaemonErrorContext adds some context to log messages to make it
* easier to associate them with the maintenance daemon.
*/
static void
MaintenanceDaemonErrorContext(void *arg)
{
MaintenanceDaemonDBData *myDbData = (MaintenanceDaemonDBData *) arg;
errcontext("Citus maintenance daemon for database %u user %u",
myDbData->databaseOid, myDbData->userOid);
}
/*
* LockCitusExtension acquires a lock on the Citus extension or returns
* false if the extension does not exist or is being dropped.
*/
static bool
LockCitusExtension(void)
{
Oid extensionOid = get_extension_oid("citus", true);
if (extensionOid == InvalidOid)
{
/* citus extension does not exist */
return false;
}
LockDatabaseObject(ExtensionRelationId, extensionOid, 0, AccessShareLock);
/*
* The extension may have been dropped and possibly recreated prior to
* obtaining a lock. Check whether we still get the expected OID.
*/
Oid recheckExtensionOid = get_extension_oid("citus", true);
if (recheckExtensionOid != extensionOid)
{
return false;
}
return true;
}
/*
* StopMaintenanceDaemon stops the maintenance daemon for the
* given database and removes it from the maintenance daemon
* control hash.
*/
void
StopMaintenanceDaemon(Oid databaseId)
{
bool found = false;
pid_t workerPid = 0;
LWLockAcquire(&MaintenanceDaemonControl->lock, LW_EXCLUSIVE);
MaintenanceDaemonDBData *dbData = (MaintenanceDaemonDBData *) hash_search(
MaintenanceDaemonDBHash,
&databaseId,
HASH_REMOVE, &found);
if (found)
{
workerPid = dbData->workerPid;
}
LWLockRelease(&MaintenanceDaemonControl->lock);
if (workerPid > 0)
{
kill(workerPid, SIGTERM);
}
}
/*
* TriggerMetadataSync triggers the maintenance daemon to do a metadata sync for
* the given database.
*/
void
TriggerMetadataSync(Oid databaseId)
{
bool found = false;
LWLockAcquire(&MaintenanceDaemonControl->lock, LW_EXCLUSIVE);
MaintenanceDaemonDBData *dbData = (MaintenanceDaemonDBData *) hash_search(
MaintenanceDaemonDBHash,
&databaseId,
HASH_FIND, &found);
if (found)
{
dbData->triggerMetadataSync = true;
/* set latch to wake-up the maintenance loop */
SetLatch(dbData->latch);
}
LWLockRelease(&MaintenanceDaemonControl->lock);
}
/*
* MetadataSyncTriggeredCheckAndReset checks if metadata sync has been
* triggered for the given database, and resets the flag.
*/
static bool
MetadataSyncTriggeredCheckAndReset(MaintenanceDaemonDBData *dbData)
{
LWLockAcquire(&MaintenanceDaemonControl->lock, LW_EXCLUSIVE);
bool metadataSyncTriggered = dbData->triggerMetadataSync;
dbData->triggerMetadataSync = false;
LWLockRelease(&MaintenanceDaemonControl->lock);
return metadataSyncTriggered;
}