Re: POC: Cleaning up orphaned files using undo logs

Поиск
Список
Период
Сортировка
От Andres Freund
Тема Re: POC: Cleaning up orphaned files using undo logs
Дата
Msg-id 20190806211842.xqof4cl2qrkwz6l2@alap3.anarazel.de
обсуждение исходный текст
Ответ на Re: POC: Cleaning up orphaned files using undo logs  (Andres Freund <andres@anarazel.de>)
Ответы Re: POC: Cleaning up orphaned files using undo logs  (Thomas Munro <thomas.munro@gmail.com>)
Re: POC: Cleaning up orphaned files using undo logs  (Andres Freund <andres@anarazel.de>)
Список pgsql-hackers
Hi,

On 2019-08-06 00:56:26 -0700, Andres Freund wrote:
> Out of energy.

Here's the last section of my low-leve review. Plan to write a higher
level summary afterwards, now that I have a better picture of the code.


> +static void
> +UndoDiscardOneLog(UndoLogSlot *slot, TransactionId xmin, bool *hibernate)

I think the naming here is pretty confusing.  We have UndoDiscard(),
UndoDiscardOneLog(), UndoLogDiscard(). I don't think anybody really can
be expected to understand what is supposed to be what from these names.


> +    /* Loop until we run out of discardable transactions in the given log. */
> +    do
> +    {

for(;;) or while (true)


> +        TransactionId wait_xid = InvalidTransactionId;
> +        bool pending_abort = false;
> +        bool request_rollback = false;
> +        UndoStatus status;
> +        UndoRecordFetchContext    context;
> +
> +        next_insert = UndoLogGetNextInsertPtr(logno);
> +
> +        /* There must be some undo data for a transaction. */
> +        Assert(next_insert != undo_recptr);
> +
> +        /* Fetch the undo record for the given undo_recptr. */
> +        BeginUndoFetch(&context);
> +        uur = UndoFetchRecord(&context, undo_recptr);
> +        FinishUndoFetch(&context);
> +
> +        if (uur != NULL)
> +        {
> +            if (UndoRecPtrGetCategory(undo_recptr) == UNDO_SHARED)

FWIW, this is precisely my problem with exposing such small
informational functions, which actually have to perform some work. As is
there's several places looking up the underlying undo slot, within just
these lines of code.

We do it once in UndoLogGetNextInsertPtr(). Then again in
UndoFetchRecord(). And then again in UndoRecPtrGetCategory(). And then
later again multiple times when actually discarding.  That perhaps
doesn't matter from a performance POV, but for me that indicates that
the APIs aren't quite right.


> +            {
> +                /*
> +                 * For the "shared" category, we only discard when the
> +                 * rm_undo_status callback tells us we can.
> +                 */

Is there a description as to what the rm_status callback is intended to
do? It currently is mandatory, is that intended?  Why does this only
apply to shared records? And why just for SHARED, not for any of the others?



> +            else
> +            {
> +                TransactionId xid = XidFromFullTransactionId(uur->uur_fxid);
> +
> +                /*
> +                 * Otherwise we use the CLOG and xmin to decide whether to
> +                 * wait, discard or roll back.
> +                 *
> +                 * XXX: We've added the transaction-in-progress check to
> +                 * avoid xids of in-progress autovacuum as those are not
> +                 * computed for oldestxmin calculation.

Hm. xids of autovacuum?  The concern here is the xid that autovacuum
might acquire when locking a relation for truncating a table at the end,
with wal_level=replica?  Because otherwise it shouldn't have any xids?



> See
> +                 * DiscardWorkerMain.

Hm. This actually reminds me of a complaint I have about this. ISTM that
the logic for discarding itself should be separate from the discard
worker. I'd just add that, and a UDF to invoke it, in a separate commit.



> +            /*
> +             * Add the aborted transaction to the rollback request queues.
> +             *
> +             * We can ignore the abort for transactions whose corresponding
> +             * database doesn't exist.
> +             */
> +            if (request_rollback && dbid_exists(uur->uur_txn->urec_dbid))
> +            {
> +                (void) RegisterUndoRequest(InvalidUndoRecPtr,
> +                                           undo_recptr,
> +                                           uur->uur_txn->urec_dbid,
> +                                           uur->uur_fxid);
> +
> +                pending_abort = true;
> +            }

As I, I think, said before: This imo should not be necessary.


> +
> +        /*
> +         * We can discard upto this point when one of following conditions is

*up to


> +         * met: (a) we need to wait for a transaction first. (b) there is no
> +         * more log to process. (c) the transaction undo in current log is
> +         * finished. (d) there is a pending abort.
> +         */

This comment is hard to understand. Perhaps you're missing some words?
Because it's e.g. not clear what it means that "we can discard up to
this point", when we "need to wait for a transaction firts". Those seem
strictly contradictory.  I assume what this is trying to say is that we
now have reached the end of the range of undo that can be discarded, so
we should do so now?  But it's really quite muddled, because we don't
actually necessarily discard here, because we might have a wait_xid, for
example?


> +        if (TransactionIdIsValid(wait_xid) ||
> +            next_urecptr == InvalidUndoRecPtr ||
> +            UndoRecPtrGetLogNo(next_urecptr) != logno ||
> +            pending_abort)

Hm. Is it guaranteed that wait_xid isn't actually old enough that we
could discard further? I haven't figured out what precisely the purpose
of rm_undo_status is, so I'm not sure. But the alternative seems to be
that the callback would need to perform its own GetOldestXmin()
computations etc, which seems to make no sense?

It seems to me that the whole DidCommit/!InProgress/ block should not be
part of the if (UndoRecPtrGetCategory(undo_recptr) == UNDO_SHARED) else
if block, but follow it? I.e. the only thing inside the else should be
XidFromFullTransactionId(uur->uur_fxid), and then we check afterwards
whether it, or rm_undo_status()'s return value requires waiting?


> +        {
> +            /* Hey, I got some undo log to discard, can not hibernate now. */
> +            *hibernate = false;

I don't understand why this block sets *hibernate to false.  I mean
need_discard is not guranteed to be true at this point, no?


> +            /*
> +             * If we don't need to wait for this transaction and this is not
> +             * an aborted transaction, then we can discard it as well.
> +             */
> +            if (!TransactionIdIsValid(wait_xid) && !pending_abort)
> +            {
> +                /*
> +                 * It is safe to use next_insert as the location till which we
> +                 * want to discard in this case.  If something new has been
> +                 * added after we have fetched this transaction's record, it
> +                 * won't be considered in this pass of discard.
> +                 */
> +                undo_recptr = next_insert;
> +                latest_discardxid = XidFromFullTransactionId(undofxid);
> +                need_discard = true;
> +
> +                /* We don't have anything more to discard. */
> +                undofxid = InvalidFullTransactionId;
> +            }
> +            /* Update the shared memory state. */
> +            LWLockAcquire(&slot->discard_lock, LW_EXCLUSIVE);
> +
> +            /*
> +             * If the slot has been recycling while we were thinking about it,

*recycled

> +             * we have to abandon the operation.
> +             */
> +            if (slot->logno != logno)
> +            {
> +                LWLockRelease(&slot->discard_lock);
> +                break;
> +            }
> +
> +            /* Update the slot information for the next pass of discard. */
> +            slot->wait_fxmin = undofxid;
> +            slot->oldest_data = undo_recptr;

Perhaps 'next pass of UndoDiscard()' instead? I found it confusing that
UndoDiscardLog() is a loop, meaning that the "next pass" could perhaps
reference the next pass through UndoDiscardLog()'s loop. But it's for
UndoDiscard().



> +            LWLockRelease(&slot->discard_lock);
> +
> +            if (need_discard)
> +            {
> +                LWLockAcquire(&slot->discard_update_lock, LW_EXCLUSIVE);
> +                UndoLogDiscard(undo_recptr, latest_discardxid);
> +                LWLockRelease(&slot->discard_update_lock);
> +            }
> +
> +            break;
> +        }

It seems to me that the entire block above just shouldn't be inside the
loop. As far as I can tell the point of the loop is to figure out up to
where we can discard.  Putting the actually discarding inside that loop
is just confusing (and requires deeper indentation than necessary).



> +/*
> + * Scan all the undo logs and register the aborted transactions.  This is
> + * called as a first function from the discard worker and only after this pass

"a first function"? There can only be one first function, no? Also, what
does "first function" really mean?

As I write earlier, I think this function name is too generic, it
doesn't explain anything. And I think it's not OK for it to be called
(the bgworker is started with BgWorkerStart_RecoveryFinished) after the
system is supposed to be ready (i.e. StartupXlog() has finished, we
signal that we're up to pg_ctl etc, and allow writing transaction), but
necessary for allowing writes to be allowed (we throw errors in
PrepareUndoInsert()).


> + * over undo logs is complete, new undo can is allowed to be written in the

"undo can"?


> + * system.  This is required because after crash recovery we don't know the
> + * exact number of aborted transactions whose rollback request is pending and
> + * we can not allow new undo request if we already have the request equal to
> + * hash table size.  So before start allowing any new transaction to write the
> + * undo we need to make sure that we know exact number of pending requests.
> + */
> +void
> +UndoLogProcess()

(void)

> +{
> +    UndoLogSlot *slot = NULL;
> +
> +    /*
> +     * We need to perform this in a transaction because (a) we need resource
> +     * owner to scan the logs and (b) TransactionIdIsInProgress requires us to
> +     * be in transaction.
> +     */
> +    StartTransactionCommand();

The need for resowners does not imply needing transactions. I think
nearly all aux processes, for example, don't use transactions, but do
have a resowner.



> +    /*
> +     * Loop through all the valid undo logs and scan them transaction by
> +     * transaction to find non-commited transactions if any and register them
> +     * in the rollback hash table.
> +     */
> +    while ((slot = UndoLogNextSlot(slot)))
> +    {
> +        UndoRecPtr    undo_recptr;
> +        UnpackedUndoRecord    *uur = NULL;
> +
> +        /* We do not execute shared (non-transactional) undo records. */
> +        if (slot->meta.category == UNDO_SHARED)
> +            continue;
> +
> +        /* Start scanning the log from the last discard point. */
> +        undo_recptr = UndoLogGetOldestRecord(slot->logno, NULL);
> +
> +        /* Loop until we scan complete log. */
> +        while (1)
> +        {
> +            TransactionId xid;
> +            UndoRecordFetchContext    context;
> +
> +            /* Done with this log. */
> +            if (!UndoRecPtrIsValid(undo_recptr))
> +                break;

Why isn't this loop while(UndoRecPtrIsValid(undo_recptr))?


> +            /*
> +             * Register the rollback request for all uncommitted and not in
> +             * progress transactions whose undo apply progress is still not
> +             * completed.  Even though we don't allow any new transactions to
> +             * write undo until this first pass is completed, there might be
> +             * some prepared transactions which are still in progress, so we
> +             * don't include such transactions.
> +             */
> +            if (!TransactionIdDidCommit(xid) &&
> +                !TransactionIdIsInProgress(xid) &&
> +                !IsXactApplyProgressCompleted(uur->uur_txn->urec_progress))
> +            {
> +                (void) RegisterUndoRequest(InvalidUndoRecPtr, undo_recptr,
> +                                           uur->uur_txn->urec_dbid,
> +                                           uur->uur_fxid);
> +            }
> +
> +            /*
> +             * Go to the next transaction in the same log.  If uur_next is
> +             * point to the undo record pointer in the different log then we are

"is point"

> +             * done with this log so just set undo_recptr to InvalidUndoRecPtr.
> +             */
> +            if (UndoRecPtrGetLogNo(undo_recptr) ==
> +                UndoRecPtrGetLogNo(uur->uur_txn->urec_next))
> +                undo_recptr = uur->uur_txn->urec_next;
> +            else
> +                undo_recptr = InvalidUndoRecPtr;
> +
> +            /* Release memory for the current record. */
> +            UndoRecordRelease(uur);
> +        }
> +    }




> +     * XXX Ideally we can arrange undo logs so that we can efficiently find
> +     * those with oldest_xid < oldestXmin, but for now we'll just scan all of
> +     * them.
> +     */
> +    while ((slot = UndoLogNextSlot(slot)))
> +    {
> +        /*
> +         * If the log is already discarded, then we are done.  It is important
> +         * to first check this to ensure that tablespace containing this log
> +         * doesn't get dropped concurrently.
> +         */
> +        LWLockAcquire(&slot->mutex, LW_SHARED);
> +        /*
> +         * We don't have to worry about slot recycling and check the logno
> +         * here, since we don't care about the identity of this slot, we're
> +         * visiting all of them.
> +         */
> +        if (slot->meta.discard == slot->meta.unlogged.insert)
> +        {
> +            LWLockRelease(&slot->mutex);
> +            continue;
> +        }
> +        LWLockRelease(&slot->mutex);

I'm fairly sure that pgindent will add some newlines here... It's a good
practice to re-pgindent patches.


> +        /* We can't process temporary undo logs. */
> +        if (slot->meta.category == UNDO_TEMP)
> +            continue;
> +
> +        /*
> +         * If the first xid of the undo log is smaller than the xmin then try
> +         * to discard the undo log.
> +         */
> +        if (!FullTransactionIdIsValid(slot->wait_fxmin) ||
> +            FullTransactionIdPrecedes(slot->wait_fxmin, oldestXidHavingUndo))

So the comment describes something different than what's happening,
while otherwise not adding much over the code... That's imo confusing.


> +        {
> +            /* Process the undo log. */
> +            UndoDiscardOneLog(slot, oldestXmin, hibernate);

That comment seems unhelpful.



> +     * XXX: In future, if multiple workers can perform discard then we may
> +     * need to use compare and swap for updating the shared memory value.
> +     */
> +    if (FullTransactionIdIsValid(oldestXidHavingUndo))
> +        pg_atomic_write_u64(&ProcGlobal->oldestFullXidHavingUnappliedUndo,
> +                            U64FromFullTransactionId(oldestXidHavingUndo));

Seems like a lock would be more appropriate if we ever needed that -
only other discard workers would need it, so ...


> +/*
> + * Discard all the logs.  This is particularly required in single user mode
> + * where at the commit time we discard all the undo logs.
> + */
> +void
> +UndoLogDiscardAll(void)
> +{
> +    UndoLogSlot *slot = NULL;
> +
> +    Assert(!IsUnderPostmaster);
> +
> +    /*
> +     * No locks are required for discard, since this called only in single
> +     * user mode.
> +     */
> +    while ((slot = UndoLogNextSlot(slot)))
> +    {
> +        /* If the log is already discarded, then we are done. */
> +        if (slot->meta.discard == slot->meta.unlogged.insert)
> +            continue;
> +
> +        /*
> +         * Process the undo log.
> +         */
> +        UndoLogDiscard(MakeUndoRecPtr(slot->logno, slot->meta.unlogged.insert),
> +                       InvalidTransactionId);
> +    }
> +
> +}

Uh. So. What happens if we start up in single user mode while
transactions that haven't been rolled back yet exist? Which seems like a
pretty typical situation for single user mode, because usually something
has gone wrong before, which means it's quite likely that there are
transactions that effectively aborted and haven't processed undo?  How
is this not entirely broken?


> +/*
> + * Discard the undo logs for temp tables.
> + */
> +void
> +TempUndoDiscard(UndoLogNumber logno)
> +{

The only callsite for this is:

+            case ONCOMMIT_TEMP_DISCARD:
+                /* Discard temp table undo logs for temp tables. */
+                TempUndoDiscard(oc->relid);
+                break;

Which looks mightily odd, given that relid doesn't really sound like an
undo log number.  There's also no code actually registering an
ONCOMMIT_TEMP_DISCARD callback.

Nor is it clear to me why it, in general, would be correct to drop undo
pre-commit, even for temp relations. It's fine for ON COMMIT DROP
relations, but what about temporary relations that are longer lived than
that? As the transaction can still fail at this stage - e.g. due to
serialization failures - we'd just throw undo away that we'll need later?



> @@ -943,9 +1077,24 @@ CanPushReqToUndoWorker(UndoRecPtr start_urec_ptr, UndoRecPtr end_urec_ptr,
>  
>      /*
>       * We normally push the rollback request to undo workers if the size of
> -     * same is above a certain threshold.
> +     * same is above a certain threshold.  However, discard worker is allowed

*the discard worker



>           * The request can't be pushed into the undo worker queue.  The

I don't think 'undo worker queue' is really correct. It's not one
worker, and it's not one queue. And we're not queueing for a specific
worker.


> -         * backends will try executing by itself.

"Executing by itself" doesn't sound right. Execute the undo itself?


> +         * backends will try executing by itself.  The discard worker will
> +         * keep the entry into the rollback hash table with

"will keep the entry into" doesn't sound right. Insert?


> +         * UNDO_REQUEST_INVALID status.  Such requests will be added in the
> +         * undo worker queues in the subsequent passes over undo logs by
> +         * discard worker.
>           */
> -        else
> +        else if (!IsDiscardProcess())
>              rh->status = UNDO_REQUEST_INPROGRESS;
> +        else
> +            rh->status = UNDO_REQUEST_INVALID;
>      }

I don't understand what the point of this is. We add an entry into the
hashtable, but mark it as invalid? How does this not allow to run out of
memory?


> + * To know more about work queues, see undorequest.c.  The worker is launched
> + * to handle requests for a particular database.

I thought we had agreed that workers pick databases after they're
started?  There seems to be plenty code in here that does not implement
that.


> +/* SIGTERM: set flag to exit at next convenient time */
> +static void
> +UndoworkerSigtermHandler(SIGNAL_ARGS)
> +{
> +    got_SIGTERM = true;
> +
> +    /* Waken anything waiting on the process latch */
> +    SetLatch(MyLatch);
> +}
> +
> +/* SIGHUP: set flag to reload configuration at next convenient time */
> +static void
> +UndoLauncherSighup(SIGNAL_ARGS)
> +{
> +    int            save_errno = errno;
> +
> +    got_SIGHUP = true;
> +
> +    /* Waken anything waiting on the process latch */
> +    SetLatch(MyLatch);
> +
> +    errno = save_errno;
> +}

So one handler saves errno, the other doesn't...


> +/*
> + * Wait for a background worker to start up and attach to the shmem context.
> + *
> + * This is only needed for cleaning up the shared memory in case the worker
> + * fails to attach.
> + */
> +static void
> +WaitForUndoWorkerAttach(UndoApplyWorker * worker,
> +                        uint16 generation,
> +                        BackgroundWorkerHandle *handle)

Once we have undo workers pick their db, this should not be needed
anymore. The launcher shouldn't even prepare anything in shared memory
for it.


> +/*
> + * Returns whether an undo worker is available.
> + */
> +static int
> +IsUndoWorkerAvailable(void)
> +{
> +    int            i;
> +    int            alive_workers = 0;
> +
> +    LWLockAcquire(UndoWorkerLock, LW_EXCLUSIVE);
> +
> +    /* Search for attached workers. */
> +    for (i = 0; i < max_undo_workers; i++)
> +    {
> +        UndoApplyWorker *w = &UndoApplyCtx->workers[i];
> +
> +        if (w->in_use)
> +            alive_workers++;
> +    }
> +
> +    LWLockRelease(UndoWorkerLock);
> +
> +    return (alive_workers < max_undo_workers);
> +}
> +
> +/* Sets the worker's lingering status. */
> +static void
> +UndoWorkerIsLingering(bool sleep)
> +{
> +    /* Block concurrent access. */
> +    LWLockAcquire(UndoWorkerLock, LW_EXCLUSIVE);
> +
> +    MyUndoWorker->lingering = sleep;
> +
> +    LWLockRelease(UndoWorkerLock);
> +}
> +
> +/* Get the dbid and undo worker queue set by the undo launcher. */
> +static void
> +UndoWorkerGetSlotInfo(int slot, UndoRequestInfo *urinfo)
> +{
> +    /* Block concurrent access. */
> +    LWLockAcquire(UndoWorkerLock, LW_EXCLUSIVE);
> +
> +    MyUndoWorker = &UndoApplyCtx->workers[slot];
> +
> +    if (!MyUndoWorker->in_use)
> +    {
> +        LWLockRelease(UndoWorkerLock);
> +        ereport(ERROR,
> +                (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
> +                 errmsg("undo worker slot %d is empty",
> +                        slot)));
> +    }
> +
> +    urinfo->dbid = MyUndoWorker->dbid;
> +    urinfo->undo_worker_queue = MyUndoWorker->undo_worker_queue;
> +
> +    LWLockRelease(UndoWorkerLock);
> +}

Why do all these need an exclusive lock?





> +/*
> + * Perform rollback request.  We need to connect to the database for first
> + * request and that is required because we access system tables while
> + * performing undo actions.
> + */
> +static void
> +UndoWorkerPerformRequest(UndoRequestInfo * urinfo)
> +{
> +    bool error = false;
> +
> +    /* must be connected to the database. */
> +    Assert(MyDatabaseId != InvalidOid);

The comment above says "We need to connect to the database", yet we
assert here that we "must be connected to the database".


> +/*
> + * UndoLauncherRegister
> + *        Register a background worker running the undo worker launcher.
> + */
> +void
> +UndoLauncherRegister(void)
> +{
> +    BackgroundWorker bgw;
> +
> +    if (max_undo_workers == 0)
> +        return;
> +
> +    memset(&bgw, 0, sizeof(bgw));
> +    bgw.bgw_flags = BGWORKER_SHMEM_ACCESS |
> +        BGWORKER_BACKEND_DATABASE_CONNECTION;
> +    bgw.bgw_start_time = BgWorkerStart_RecoveryFinished;
> +    snprintf(bgw.bgw_library_name, BGW_MAXLEN, "postgres");
> +    snprintf(bgw.bgw_function_name, BGW_MAXLEN, "UndoLauncherMain");
> +    snprintf(bgw.bgw_name, BGW_MAXLEN,
> +             "undo worker launcher");
> +    snprintf(bgw.bgw_type, BGW_MAXLEN,
> +             "undo worker launcher");
> +    bgw.bgw_restart_time = 5;
> +    bgw.bgw_notify_pid = 0;
> +    bgw.bgw_main_arg = (Datum)0;
> +
> +    RegisterBackgroundWorker(&bgw);
> +}
> +
> +/*
> + * Main loop for the undo worker launcher process.
> + */
> +void
> +UndoLauncherMain(Datum main_arg)
> +{
> +    UndoRequestInfo urinfo;
> +
> +    ereport(DEBUG1,
> +            (errmsg("undo launcher started")));
> +
> +    before_shmem_exit(UndoLauncherOnExit, (Datum) 0);
> +
> +    Assert(UndoApplyCtx->launcher_pid == 0);
> +    UndoApplyCtx->launcher_pid = MyProcPid;
> +
> +    /* Establish signal handlers. */
> +    pqsignal(SIGHUP, UndoLauncherSighup);
> +    pqsignal(SIGTERM, UndoworkerSigtermHandler);
> +    BackgroundWorkerUnblockSignals();
> +
> +    /* Establish connection to nailed catalogs. */
> +    BackgroundWorkerInitializeConnection(NULL, NULL, 0);

Why do we need to be connected in the launcher? I assume that's because
we still do checks on the database?



Greetings,

Andres Freund



В списке pgsql-hackers по дате отправления:

Предыдущее
От: "Joshua D. Drake"
Дата:
Сообщение: Re: Cleanup of intro.sgml
Следующее
От: Stephen Frost
Дата:
Сообщение: Re: [PATCH] Stop ALTER SYSTEM from making bad assumptions