mirror of
https://github.com/postgres/postgres.git
synced 2025-11-19 13:42:17 +03:00
Support parallel btree index builds.
To make this work, tuplesort.c and logtape.c must also support parallelism, so this patch adds that infrastructure and then applies it to the particular case of parallel btree index builds. Testing to date shows that this can often be 2-3x faster than a serial index build. The model for deciding how many workers to use is fairly primitive at present, but it's better than not having the feature. We can refine it as we get more experience. Peter Geoghegan with some help from Rushabh Lathia. While Heikki Linnakangas is not an author of this patch, he wrote other patches without which this feature would not have been possible, and therefore the release notes should possibly credit him as an author of this feature. Reviewed by Claudio Freire, Heikki Linnakangas, Thomas Munro, Tels, Amit Kapila, me. Discussion: http://postgr.es/m/CAM3SWZQKM=Pzc=CAHzRixKjp2eO5Q0Jg1SoFQqeXFQ647JiwqQ@mail.gmail.com Discussion: http://postgr.es/m/CAH2-Wz=AxWqDoVvGU7dq856S4r6sJAj6DBn7VMtigkB33N5eyg@mail.gmail.com
This commit is contained in:
@@ -64,6 +64,14 @@
|
||||
* care that all calls for a single LogicalTapeSet are made in the same
|
||||
* palloc context.
|
||||
*
|
||||
* To support parallel sort operations involving coordinated callers to
|
||||
* tuplesort.c routines across multiple workers, it is necessary to
|
||||
* concatenate each worker BufFile/tapeset into one single logical tapeset
|
||||
* managed by the leader. Workers should have produced one final
|
||||
* materialized tape (their entire output) when this happens in leader.
|
||||
* There will always be the same number of runs as input tapes, and the same
|
||||
* number of input tapes as participants (worker Tuplesortstates).
|
||||
*
|
||||
* Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
|
||||
* Portions Copyright (c) 1994, Regents of the University of California
|
||||
*
|
||||
@@ -76,6 +84,7 @@
|
||||
#include "postgres.h"
|
||||
|
||||
#include "storage/buffile.h"
|
||||
#include "utils/builtins.h"
|
||||
#include "utils/logtape.h"
|
||||
#include "utils/memutils.h"
|
||||
|
||||
@@ -129,16 +138,21 @@ typedef struct LogicalTape
|
||||
* a frozen tape. (When reading from an unfrozen tape, we use a larger
|
||||
* read buffer that holds multiple blocks, so the "current" block is
|
||||
* ambiguous.)
|
||||
*
|
||||
* When concatenation of worker tape BufFiles is performed, an offset to
|
||||
* the first block in the unified BufFile space is applied during reads.
|
||||
*/
|
||||
long firstBlockNumber;
|
||||
long curBlockNumber;
|
||||
long nextBlockNumber;
|
||||
long offsetBlockNumber;
|
||||
|
||||
/*
|
||||
* Buffer for current data block(s).
|
||||
*/
|
||||
char *buffer; /* physical buffer (separately palloc'd) */
|
||||
int buffer_size; /* allocated size of the buffer */
|
||||
int max_size; /* highest useful, safe buffer_size */
|
||||
int pos; /* next read/write position in buffer */
|
||||
int nbytes; /* total # of valid bytes in buffer */
|
||||
} LogicalTape;
|
||||
@@ -159,10 +173,13 @@ struct LogicalTapeSet
|
||||
* by ltsGetFreeBlock(), and it is always greater than or equal to
|
||||
* nBlocksWritten. Blocks between nBlocksAllocated and nBlocksWritten are
|
||||
* blocks that have been allocated for a tape, but have not been written
|
||||
* to the underlying file yet.
|
||||
* to the underlying file yet. nHoleBlocks tracks the total number of
|
||||
* blocks that are in unused holes between worker spaces following BufFile
|
||||
* concatenation.
|
||||
*/
|
||||
long nBlocksAllocated; /* # of blocks allocated */
|
||||
long nBlocksWritten; /* # of blocks used in underlying file */
|
||||
long nHoleBlocks; /* # of "hole" blocks left */
|
||||
|
||||
/*
|
||||
* We store the numbers of recycled-and-available blocks in freeBlocks[].
|
||||
@@ -192,6 +209,8 @@ static void ltsWriteBlock(LogicalTapeSet *lts, long blocknum, void *buffer);
|
||||
static void ltsReadBlock(LogicalTapeSet *lts, long blocknum, void *buffer);
|
||||
static long ltsGetFreeBlock(LogicalTapeSet *lts);
|
||||
static void ltsReleaseBlock(LogicalTapeSet *lts, long blocknum);
|
||||
static void ltsConcatWorkerTapes(LogicalTapeSet *lts, TapeShare *shared,
|
||||
SharedFileSet *fileset);
|
||||
|
||||
|
||||
/*
|
||||
@@ -213,6 +232,11 @@ ltsWriteBlock(LogicalTapeSet *lts, long blocknum, void *buffer)
|
||||
* previous tape isn't flushed to disk until the end of the sort, so you
|
||||
* get one-block hole, where the last block of the previous tape will
|
||||
* later go.
|
||||
*
|
||||
* Note that BufFile concatenation can leave "holes" in BufFile between
|
||||
* worker-owned block ranges. These are tracked for reporting purposes
|
||||
* only. We never read from nor write to these hole blocks, and so they
|
||||
* are not considered here.
|
||||
*/
|
||||
while (blocknum > lts->nBlocksWritten)
|
||||
{
|
||||
@@ -267,15 +291,18 @@ ltsReadFillBuffer(LogicalTapeSet *lts, LogicalTape *lt)
|
||||
do
|
||||
{
|
||||
char *thisbuf = lt->buffer + lt->nbytes;
|
||||
long datablocknum = lt->nextBlockNumber;
|
||||
|
||||
/* Fetch next block number */
|
||||
if (lt->nextBlockNumber == -1L)
|
||||
if (datablocknum == -1L)
|
||||
break; /* EOF */
|
||||
/* Apply worker offset, needed for leader tapesets */
|
||||
datablocknum += lt->offsetBlockNumber;
|
||||
|
||||
/* Read the block */
|
||||
ltsReadBlock(lts, lt->nextBlockNumber, (void *) thisbuf);
|
||||
ltsReadBlock(lts, datablocknum, (void *) thisbuf);
|
||||
if (!lt->frozen)
|
||||
ltsReleaseBlock(lts, lt->nextBlockNumber);
|
||||
ltsReleaseBlock(lts, datablocknum);
|
||||
lt->curBlockNumber = lt->nextBlockNumber;
|
||||
|
||||
lt->nbytes += TapeBlockGetNBytes(thisbuf);
|
||||
@@ -370,13 +397,116 @@ ltsReleaseBlock(LogicalTapeSet *lts, long blocknum)
|
||||
lts->blocksSorted = false;
|
||||
}
|
||||
|
||||
/*
|
||||
* Claim ownership of a set of logical tapes from existing shared BufFiles.
|
||||
*
|
||||
* Caller should be leader process. Though tapes are marked as frozen in
|
||||
* workers, they are not frozen when opened within leader, since unfrozen tapes
|
||||
* use a larger read buffer. (Frozen tapes have smaller read buffer, optimized
|
||||
* for random access.)
|
||||
*/
|
||||
static void
|
||||
ltsConcatWorkerTapes(LogicalTapeSet *lts, TapeShare *shared,
|
||||
SharedFileSet *fileset)
|
||||
{
|
||||
LogicalTape *lt = NULL;
|
||||
long tapeblocks;
|
||||
long nphysicalblocks = 0L;
|
||||
int i;
|
||||
|
||||
/* Should have at least one worker tape, plus leader's tape */
|
||||
Assert(lts->nTapes >= 2);
|
||||
|
||||
/*
|
||||
* Build concatenated view of all BufFiles, remembering the block number
|
||||
* where each source file begins. No changes are needed for leader/last
|
||||
* tape.
|
||||
*/
|
||||
for (i = 0; i < lts->nTapes - 1; i++)
|
||||
{
|
||||
char filename[MAXPGPATH];
|
||||
BufFile *file;
|
||||
|
||||
lt = <s->tapes[i];
|
||||
|
||||
pg_itoa(i, filename);
|
||||
file = BufFileOpenShared(fileset, filename);
|
||||
|
||||
/*
|
||||
* Stash first BufFile, and concatenate subsequent BufFiles to that.
|
||||
* Store block offset into each tape as we go.
|
||||
*/
|
||||
lt->firstBlockNumber = shared[i].firstblocknumber;
|
||||
if (i == 0)
|
||||
{
|
||||
lts->pfile = file;
|
||||
lt->offsetBlockNumber = 0L;
|
||||
}
|
||||
else
|
||||
{
|
||||
lt->offsetBlockNumber = BufFileAppend(lts->pfile, file);
|
||||
}
|
||||
/* Don't allocate more for read buffer than could possibly help */
|
||||
lt->max_size = Min(MaxAllocSize, shared[i].buffilesize);
|
||||
tapeblocks = shared[i].buffilesize / BLCKSZ;
|
||||
nphysicalblocks += tapeblocks;
|
||||
}
|
||||
|
||||
/*
|
||||
* Set # of allocated blocks, as well as # blocks written. Use extent of
|
||||
* new BufFile space (from 0 to end of last worker's tape space) for this.
|
||||
* Allocated/written blocks should include space used by holes left
|
||||
* between concatenated BufFiles.
|
||||
*/
|
||||
lts->nBlocksAllocated = lt->offsetBlockNumber + tapeblocks;
|
||||
lts->nBlocksWritten = lts->nBlocksAllocated;
|
||||
|
||||
/*
|
||||
* Compute number of hole blocks so that we can later work backwards, and
|
||||
* instrument number of physical blocks. We don't simply use physical
|
||||
* blocks directly for instrumentation because this would break if we ever
|
||||
* subsequently wrote to worker tape.
|
||||
*
|
||||
* Working backwards like this keeps our options open. If shared BufFiles
|
||||
* ever support being written to post-export, logtape.c can automatically
|
||||
* take advantage of that. We'd then support writing to the leader tape
|
||||
* while recycling space from worker tapes, because the leader tape has a
|
||||
* zero offset (write routines won't need to have extra logic to apply an
|
||||
* offset).
|
||||
*
|
||||
* The only thing that currently prevents writing to the leader tape from
|
||||
* working is the fact that BufFiles opened using BufFileOpenShared() are
|
||||
* read-only by definition, but that could be changed if it seemed
|
||||
* worthwhile. For now, writing to the leader tape will raise a "Bad file
|
||||
* descriptor" error, so tuplesort must avoid writing to the leader tape
|
||||
* altogether.
|
||||
*/
|
||||
lts->nHoleBlocks = lts->nBlocksAllocated - nphysicalblocks;
|
||||
}
|
||||
|
||||
/*
|
||||
* Create a set of logical tapes in a temporary underlying file.
|
||||
*
|
||||
* Each tape is initialized in write state.
|
||||
* Each tape is initialized in write state. Serial callers pass ntapes,
|
||||
* NULL argument for shared, and -1 for worker. Parallel worker callers
|
||||
* pass ntapes, a shared file handle, NULL shared argument, and their own
|
||||
* worker number. Leader callers, which claim shared worker tapes here,
|
||||
* must supply non-sentinel values for all arguments except worker number,
|
||||
* which should be -1.
|
||||
*
|
||||
* Leader caller is passing back an array of metadata each worker captured
|
||||
* when LogicalTapeFreeze() was called for their final result tapes. Passed
|
||||
* tapes array is actually sized ntapes - 1, because it includes only
|
||||
* worker tapes, whereas leader requires its own leader tape. Note that we
|
||||
* rely on the assumption that reclaimed worker tapes will only be read
|
||||
* from once by leader, and never written to again (tapes are initialized
|
||||
* for writing, but that's only to be consistent). Leader may not write to
|
||||
* its own tape purely due to a restriction in the shared buffile
|
||||
* infrastructure that may be lifted in the future.
|
||||
*/
|
||||
LogicalTapeSet *
|
||||
LogicalTapeSetCreate(int ntapes)
|
||||
LogicalTapeSetCreate(int ntapes, TapeShare *shared, SharedFileSet *fileset,
|
||||
int worker)
|
||||
{
|
||||
LogicalTapeSet *lts;
|
||||
LogicalTape *lt;
|
||||
@@ -388,9 +518,9 @@ LogicalTapeSetCreate(int ntapes)
|
||||
Assert(ntapes > 0);
|
||||
lts = (LogicalTapeSet *) palloc(offsetof(LogicalTapeSet, tapes) +
|
||||
ntapes * sizeof(LogicalTape));
|
||||
lts->pfile = BufFileCreateTemp(false);
|
||||
lts->nBlocksAllocated = 0L;
|
||||
lts->nBlocksWritten = 0L;
|
||||
lts->nHoleBlocks = 0L;
|
||||
lts->forgetFreeSpace = false;
|
||||
lts->blocksSorted = true; /* a zero-length array is sorted ... */
|
||||
lts->freeBlocksLen = 32; /* reasonable initial guess */
|
||||
@@ -412,11 +542,36 @@ LogicalTapeSetCreate(int ntapes)
|
||||
lt->dirty = false;
|
||||
lt->firstBlockNumber = -1L;
|
||||
lt->curBlockNumber = -1L;
|
||||
lt->nextBlockNumber = -1L;
|
||||
lt->offsetBlockNumber = 0L;
|
||||
lt->buffer = NULL;
|
||||
lt->buffer_size = 0;
|
||||
/* palloc() larger than MaxAllocSize would fail */
|
||||
lt->max_size = MaxAllocSize;
|
||||
lt->pos = 0;
|
||||
lt->nbytes = 0;
|
||||
}
|
||||
|
||||
/*
|
||||
* Create temp BufFile storage as required.
|
||||
*
|
||||
* Leader concatenates worker tapes, which requires special adjustment to
|
||||
* final tapeset data. Things are simpler for the worker case and the
|
||||
* serial case, though. They are generally very similar -- workers use a
|
||||
* shared fileset, whereas serial sorts use a conventional serial BufFile.
|
||||
*/
|
||||
if (shared)
|
||||
ltsConcatWorkerTapes(lts, shared, fileset);
|
||||
else if (fileset)
|
||||
{
|
||||
char filename[MAXPGPATH];
|
||||
|
||||
pg_itoa(worker, filename);
|
||||
lts->pfile = BufFileCreateShared(fileset, filename);
|
||||
}
|
||||
else
|
||||
lts->pfile = BufFileCreateTemp(false);
|
||||
|
||||
return lts;
|
||||
}
|
||||
|
||||
@@ -470,6 +625,7 @@ LogicalTapeWrite(LogicalTapeSet *lts, int tapenum,
|
||||
Assert(tapenum >= 0 && tapenum < lts->nTapes);
|
||||
lt = <s->tapes[tapenum];
|
||||
Assert(lt->writing);
|
||||
Assert(lt->offsetBlockNumber == 0L);
|
||||
|
||||
/* Allocate data buffer and first block on first write */
|
||||
if (lt->buffer == NULL)
|
||||
@@ -566,12 +722,9 @@ LogicalTapeRewindForRead(LogicalTapeSet *lts, int tapenum, size_t buffer_size)
|
||||
if (buffer_size < BLCKSZ)
|
||||
buffer_size = BLCKSZ;
|
||||
|
||||
/*
|
||||
* palloc() larger than MaxAllocSize would fail (a multi-gigabyte
|
||||
* buffer is unlikely to be helpful, anyway)
|
||||
*/
|
||||
if (buffer_size > MaxAllocSize)
|
||||
buffer_size = MaxAllocSize;
|
||||
/* palloc() larger than max_size is unlikely to be helpful */
|
||||
if (buffer_size > lt->max_size)
|
||||
buffer_size = lt->max_size;
|
||||
|
||||
/* round down to BLCKSZ boundary */
|
||||
buffer_size -= buffer_size % BLCKSZ;
|
||||
@@ -698,15 +851,22 @@ LogicalTapeRead(LogicalTapeSet *lts, int tapenum,
|
||||
* tape is rewound (after rewind is too late!). It performs a rewind
|
||||
* and switch to read mode "for free". An immediately following rewind-
|
||||
* for-read call is OK but not necessary.
|
||||
*
|
||||
* share output argument is set with details of storage used for tape after
|
||||
* freezing, which may be passed to LogicalTapeSetCreate within leader
|
||||
* process later. This metadata is only of interest to worker callers
|
||||
* freezing their final output for leader (single materialized tape).
|
||||
* Serial sorts should set share to NULL.
|
||||
*/
|
||||
void
|
||||
LogicalTapeFreeze(LogicalTapeSet *lts, int tapenum)
|
||||
LogicalTapeFreeze(LogicalTapeSet *lts, int tapenum, TapeShare *share)
|
||||
{
|
||||
LogicalTape *lt;
|
||||
|
||||
Assert(tapenum >= 0 && tapenum < lts->nTapes);
|
||||
lt = <s->tapes[tapenum];
|
||||
Assert(lt->writing);
|
||||
Assert(lt->offsetBlockNumber == 0L);
|
||||
|
||||
/*
|
||||
* Completion of a write phase. Flush last partial data block, and rewind
|
||||
@@ -749,6 +909,14 @@ LogicalTapeFreeze(LogicalTapeSet *lts, int tapenum)
|
||||
else
|
||||
lt->nextBlockNumber = TapeBlockGetTrailer(lt->buffer)->next;
|
||||
lt->nbytes = TapeBlockGetNBytes(lt->buffer);
|
||||
|
||||
/* Handle extra steps when caller is to share its tapeset */
|
||||
if (share)
|
||||
{
|
||||
BufFileExportShared(lts->pfile);
|
||||
share->firstblocknumber = lt->firstBlockNumber;
|
||||
share->buffilesize = BufFileSize(lts->pfile);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
@@ -874,6 +1042,7 @@ LogicalTapeTell(LogicalTapeSet *lts, int tapenum,
|
||||
|
||||
Assert(tapenum >= 0 && tapenum < lts->nTapes);
|
||||
lt = <s->tapes[tapenum];
|
||||
Assert(lt->offsetBlockNumber == 0L);
|
||||
|
||||
/* With a larger buffer, 'pos' wouldn't be the same as offset within page */
|
||||
Assert(lt->buffer_size == BLCKSZ);
|
||||
@@ -888,5 +1057,5 @@ LogicalTapeTell(LogicalTapeSet *lts, int tapenum,
|
||||
long
|
||||
LogicalTapeSetBlocks(LogicalTapeSet *lts)
|
||||
{
|
||||
return lts->nBlocksAllocated;
|
||||
return lts->nBlocksAllocated - lts->nHoleBlocks;
|
||||
}
|
||||
|
||||
@@ -74,6 +74,14 @@
|
||||
* above. Nonetheless, with large workMem we can have many tapes (but not
|
||||
* too many -- see the comments in tuplesort_merge_order).
|
||||
*
|
||||
* This module supports parallel sorting. Parallel sorts involve coordination
|
||||
* among one or more worker processes, and a leader process, each with its own
|
||||
* tuplesort state. The leader process (or, more accurately, the
|
||||
* Tuplesortstate associated with a leader process) creates a full tapeset
|
||||
* consisting of worker tapes with one run to merge; a run for every
|
||||
* worker process. This is then merged. Worker processes are guaranteed to
|
||||
* produce exactly one output run from their partial input.
|
||||
*
|
||||
*
|
||||
* Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
|
||||
* Portions Copyright (c) 1994, Regents of the University of California
|
||||
@@ -113,6 +121,10 @@
|
||||
#define DATUM_SORT 2
|
||||
#define CLUSTER_SORT 3
|
||||
|
||||
/* Sort parallel code from state for sort__start probes */
|
||||
#define PARALLEL_SORT(state) ((state)->shared == NULL ? 0 : \
|
||||
(state)->worker >= 0 ? 1 : 2)
|
||||
|
||||
/* GUC variables */
|
||||
#ifdef TRACE_SORT
|
||||
bool trace_sort = false;
|
||||
@@ -374,6 +386,25 @@ struct Tuplesortstate
|
||||
int markpos_offset; /* saved "current", or offset in tape block */
|
||||
bool markpos_eof; /* saved "eof_reached" */
|
||||
|
||||
/*
|
||||
* These variables are used during parallel sorting.
|
||||
*
|
||||
* worker is our worker identifier. Follows the general convention that
|
||||
* -1 value relates to a leader tuplesort, and values >= 0 worker
|
||||
* tuplesorts. (-1 can also be a serial tuplesort.)
|
||||
*
|
||||
* shared is mutable shared memory state, which is used to coordinate
|
||||
* parallel sorts.
|
||||
*
|
||||
* nParticipants is the number of worker Tuplesortstates known by the
|
||||
* leader to have actually been launched, which implies that they must
|
||||
* finish a run leader can merge. Typically includes a worker state held
|
||||
* by the leader process itself. Set in the leader Tuplesortstate only.
|
||||
*/
|
||||
int worker;
|
||||
Sharedsort *shared;
|
||||
int nParticipants;
|
||||
|
||||
/*
|
||||
* The sortKeys variable is used by every case other than the hash index
|
||||
* case; it is set by tuplesort_begin_xxx. tupDesc is only used by the
|
||||
@@ -435,6 +466,39 @@ struct Tuplesortstate
|
||||
#endif
|
||||
};
|
||||
|
||||
/*
|
||||
* Private mutable state of tuplesort-parallel-operation. This is allocated
|
||||
* in shared memory.
|
||||
*/
|
||||
struct Sharedsort
|
||||
{
|
||||
/* mutex protects all fields prior to tapes */
|
||||
slock_t mutex;
|
||||
|
||||
/*
|
||||
* currentWorker generates ordinal identifier numbers for parallel sort
|
||||
* workers. These start from 0, and are always gapless.
|
||||
*
|
||||
* Workers increment workersFinished to indicate having finished. If this
|
||||
* is equal to state.nParticipants within the leader, leader is ready to
|
||||
* merge worker runs.
|
||||
*/
|
||||
int currentWorker;
|
||||
int workersFinished;
|
||||
|
||||
/* Temporary file space */
|
||||
SharedFileSet fileset;
|
||||
|
||||
/* Size of tapes flexible array */
|
||||
int nTapes;
|
||||
|
||||
/*
|
||||
* Tapes array used by workers to report back information needed by the
|
||||
* leader to concatenate all worker tapes into one for merging
|
||||
*/
|
||||
TapeShare tapes[FLEXIBLE_ARRAY_MEMBER];
|
||||
};
|
||||
|
||||
/*
|
||||
* Is the given tuple allocated from the slab memory arena?
|
||||
*/
|
||||
@@ -465,6 +529,9 @@ struct Tuplesortstate
|
||||
#define LACKMEM(state) ((state)->availMem < 0 && !(state)->slabAllocatorUsed)
|
||||
#define USEMEM(state,amt) ((state)->availMem -= (amt))
|
||||
#define FREEMEM(state,amt) ((state)->availMem += (amt))
|
||||
#define SERIAL(state) ((state)->shared == NULL)
|
||||
#define WORKER(state) ((state)->shared && (state)->worker != -1)
|
||||
#define LEADER(state) ((state)->shared && (state)->worker == -1)
|
||||
|
||||
/*
|
||||
* NOTES about on-tape representation of tuples:
|
||||
@@ -521,10 +588,13 @@ struct Tuplesortstate
|
||||
} while(0)
|
||||
|
||||
|
||||
static Tuplesortstate *tuplesort_begin_common(int workMem, bool randomAccess);
|
||||
static Tuplesortstate *tuplesort_begin_common(int workMem,
|
||||
SortCoordinate coordinate,
|
||||
bool randomAccess);
|
||||
static void puttuple_common(Tuplesortstate *state, SortTuple *tuple);
|
||||
static bool consider_abort_common(Tuplesortstate *state);
|
||||
static void inittapes(Tuplesortstate *state);
|
||||
static void inittapes(Tuplesortstate *state, bool mergeruns);
|
||||
static void inittapestate(Tuplesortstate *state, int maxTapes);
|
||||
static void selectnewtape(Tuplesortstate *state);
|
||||
static void init_slab_allocator(Tuplesortstate *state, int numSlots);
|
||||
static void mergeruns(Tuplesortstate *state);
|
||||
@@ -572,6 +642,10 @@ static void writetup_datum(Tuplesortstate *state, int tapenum,
|
||||
SortTuple *stup);
|
||||
static void readtup_datum(Tuplesortstate *state, SortTuple *stup,
|
||||
int tapenum, unsigned int len);
|
||||
static int worker_get_identifier(Tuplesortstate *state);
|
||||
static void worker_freeze_result_tape(Tuplesortstate *state);
|
||||
static void worker_nomergeruns(Tuplesortstate *state);
|
||||
static void leader_takeover_tapes(Tuplesortstate *state);
|
||||
static void free_sort_tuple(Tuplesortstate *state, SortTuple *stup);
|
||||
|
||||
/*
|
||||
@@ -604,13 +678,18 @@ static void free_sort_tuple(Tuplesortstate *state, SortTuple *stup);
|
||||
*/
|
||||
|
||||
static Tuplesortstate *
|
||||
tuplesort_begin_common(int workMem, bool randomAccess)
|
||||
tuplesort_begin_common(int workMem, SortCoordinate coordinate,
|
||||
bool randomAccess)
|
||||
{
|
||||
Tuplesortstate *state;
|
||||
MemoryContext sortcontext;
|
||||
MemoryContext tuplecontext;
|
||||
MemoryContext oldcontext;
|
||||
|
||||
/* See leader_takeover_tapes() remarks on randomAccess support */
|
||||
if (coordinate && randomAccess)
|
||||
elog(ERROR, "random access disallowed under parallel sort");
|
||||
|
||||
/*
|
||||
* Create a working memory context for this sort operation. All data
|
||||
* needed by the sort will live inside this context.
|
||||
@@ -650,7 +729,14 @@ tuplesort_begin_common(int workMem, bool randomAccess)
|
||||
state->bounded = false;
|
||||
state->tuples = true;
|
||||
state->boundUsed = false;
|
||||
state->allowedMem = workMem * (int64) 1024;
|
||||
|
||||
/*
|
||||
* workMem is forced to be at least 64KB, the current minimum valid value
|
||||
* for the work_mem GUC. This is a defense against parallel sort callers
|
||||
* that divide out memory among many workers in a way that leaves each
|
||||
* with very little memory.
|
||||
*/
|
||||
state->allowedMem = Max(workMem, 64) * (int64) 1024;
|
||||
state->availMem = state->allowedMem;
|
||||
state->sortcontext = sortcontext;
|
||||
state->tuplecontext = tuplecontext;
|
||||
@@ -684,6 +770,33 @@ tuplesort_begin_common(int workMem, bool randomAccess)
|
||||
|
||||
state->result_tape = -1; /* flag that result tape has not been formed */
|
||||
|
||||
/*
|
||||
* Initialize parallel-related state based on coordination information
|
||||
* from caller
|
||||
*/
|
||||
if (!coordinate)
|
||||
{
|
||||
/* Serial sort */
|
||||
state->shared = NULL;
|
||||
state->worker = -1;
|
||||
state->nParticipants = -1;
|
||||
}
|
||||
else if (coordinate->isWorker)
|
||||
{
|
||||
/* Parallel worker produces exactly one final run from all input */
|
||||
state->shared = coordinate->sharedsort;
|
||||
state->worker = worker_get_identifier(state);
|
||||
state->nParticipants = -1;
|
||||
}
|
||||
else
|
||||
{
|
||||
/* Parallel leader state only used for final merge */
|
||||
state->shared = coordinate->sharedsort;
|
||||
state->worker = -1;
|
||||
state->nParticipants = coordinate->nParticipants;
|
||||
Assert(state->nParticipants >= 1);
|
||||
}
|
||||
|
||||
MemoryContextSwitchTo(oldcontext);
|
||||
|
||||
return state;
|
||||
@@ -694,9 +807,10 @@ tuplesort_begin_heap(TupleDesc tupDesc,
|
||||
int nkeys, AttrNumber *attNums,
|
||||
Oid *sortOperators, Oid *sortCollations,
|
||||
bool *nullsFirstFlags,
|
||||
int workMem, bool randomAccess)
|
||||
int workMem, SortCoordinate coordinate, bool randomAccess)
|
||||
{
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
|
||||
randomAccess);
|
||||
MemoryContext oldcontext;
|
||||
int i;
|
||||
|
||||
@@ -717,7 +831,8 @@ tuplesort_begin_heap(TupleDesc tupDesc,
|
||||
false, /* no unique check */
|
||||
nkeys,
|
||||
workMem,
|
||||
randomAccess);
|
||||
randomAccess,
|
||||
PARALLEL_SORT(state));
|
||||
|
||||
state->comparetup = comparetup_heap;
|
||||
state->copytup = copytup_heap;
|
||||
@@ -764,9 +879,11 @@ tuplesort_begin_heap(TupleDesc tupDesc,
|
||||
Tuplesortstate *
|
||||
tuplesort_begin_cluster(TupleDesc tupDesc,
|
||||
Relation indexRel,
|
||||
int workMem, bool randomAccess)
|
||||
int workMem,
|
||||
SortCoordinate coordinate, bool randomAccess)
|
||||
{
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
|
||||
randomAccess);
|
||||
ScanKey indexScanKey;
|
||||
MemoryContext oldcontext;
|
||||
int i;
|
||||
@@ -789,7 +906,8 @@ tuplesort_begin_cluster(TupleDesc tupDesc,
|
||||
false, /* no unique check */
|
||||
state->nKeys,
|
||||
workMem,
|
||||
randomAccess);
|
||||
randomAccess,
|
||||
PARALLEL_SORT(state));
|
||||
|
||||
state->comparetup = comparetup_cluster;
|
||||
state->copytup = copytup_cluster;
|
||||
@@ -857,9 +975,12 @@ Tuplesortstate *
|
||||
tuplesort_begin_index_btree(Relation heapRel,
|
||||
Relation indexRel,
|
||||
bool enforceUnique,
|
||||
int workMem, bool randomAccess)
|
||||
int workMem,
|
||||
SortCoordinate coordinate,
|
||||
bool randomAccess)
|
||||
{
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
|
||||
randomAccess);
|
||||
ScanKey indexScanKey;
|
||||
MemoryContext oldcontext;
|
||||
int i;
|
||||
@@ -880,7 +1001,8 @@ tuplesort_begin_index_btree(Relation heapRel,
|
||||
enforceUnique,
|
||||
state->nKeys,
|
||||
workMem,
|
||||
randomAccess);
|
||||
randomAccess,
|
||||
PARALLEL_SORT(state));
|
||||
|
||||
state->comparetup = comparetup_index_btree;
|
||||
state->copytup = copytup_index;
|
||||
@@ -934,9 +1056,12 @@ tuplesort_begin_index_hash(Relation heapRel,
|
||||
uint32 high_mask,
|
||||
uint32 low_mask,
|
||||
uint32 max_buckets,
|
||||
int workMem, bool randomAccess)
|
||||
int workMem,
|
||||
SortCoordinate coordinate,
|
||||
bool randomAccess)
|
||||
{
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
|
||||
randomAccess);
|
||||
MemoryContext oldcontext;
|
||||
|
||||
oldcontext = MemoryContextSwitchTo(state->sortcontext);
|
||||
@@ -973,10 +1098,11 @@ tuplesort_begin_index_hash(Relation heapRel,
|
||||
|
||||
Tuplesortstate *
|
||||
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
|
||||
bool nullsFirstFlag,
|
||||
int workMem, bool randomAccess)
|
||||
bool nullsFirstFlag, int workMem,
|
||||
SortCoordinate coordinate, bool randomAccess)
|
||||
{
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
|
||||
Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
|
||||
randomAccess);
|
||||
MemoryContext oldcontext;
|
||||
int16 typlen;
|
||||
bool typbyval;
|
||||
@@ -996,7 +1122,8 @@ tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
|
||||
false, /* no unique check */
|
||||
1,
|
||||
workMem,
|
||||
randomAccess);
|
||||
randomAccess,
|
||||
PARALLEL_SORT(state));
|
||||
|
||||
state->comparetup = comparetup_datum;
|
||||
state->copytup = copytup_datum;
|
||||
@@ -1054,7 +1181,7 @@ tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
|
||||
* delayed calls at the moment.)
|
||||
*
|
||||
* This is a hint only. The tuplesort may still return more tuples than
|
||||
* requested.
|
||||
* requested. Parallel leader tuplesorts will always ignore the hint.
|
||||
*/
|
||||
void
|
||||
tuplesort_set_bound(Tuplesortstate *state, int64 bound)
|
||||
@@ -1063,6 +1190,7 @@ tuplesort_set_bound(Tuplesortstate *state, int64 bound)
|
||||
Assert(state->status == TSS_INITIAL);
|
||||
Assert(state->memtupcount == 0);
|
||||
Assert(!state->bounded);
|
||||
Assert(!WORKER(state));
|
||||
|
||||
#ifdef DEBUG_BOUNDED_SORT
|
||||
/* Honor GUC setting that disables the feature (for easy testing) */
|
||||
@@ -1070,6 +1198,10 @@ tuplesort_set_bound(Tuplesortstate *state, int64 bound)
|
||||
return;
|
||||
#endif
|
||||
|
||||
/* Parallel leader ignores hint */
|
||||
if (LEADER(state))
|
||||
return;
|
||||
|
||||
/* We want to be able to compute bound * 2, so limit the setting */
|
||||
if (bound > (int64) (INT_MAX / 2))
|
||||
return;
|
||||
@@ -1128,11 +1260,13 @@ tuplesort_end(Tuplesortstate *state)
|
||||
if (trace_sort)
|
||||
{
|
||||
if (state->tapeset)
|
||||
elog(LOG, "external sort ended, %ld disk blocks used: %s",
|
||||
spaceUsed, pg_rusage_show(&state->ru_start));
|
||||
elog(LOG, "%s of %d ended, %ld disk blocks used: %s",
|
||||
SERIAL(state) ? "external sort" : "parallel external sort",
|
||||
state->worker, spaceUsed, pg_rusage_show(&state->ru_start));
|
||||
else
|
||||
elog(LOG, "internal sort ended, %ld KB used: %s",
|
||||
spaceUsed, pg_rusage_show(&state->ru_start));
|
||||
elog(LOG, "%s of %d ended, %ld KB used: %s",
|
||||
SERIAL(state) ? "internal sort" : "unperformed parallel sort",
|
||||
state->worker, spaceUsed, pg_rusage_show(&state->ru_start));
|
||||
}
|
||||
|
||||
TRACE_POSTGRESQL_SORT_DONE(state->tapeset != NULL, spaceUsed);
|
||||
@@ -1503,6 +1637,8 @@ tuplesort_putdatum(Tuplesortstate *state, Datum val, bool isNull)
|
||||
static void
|
||||
puttuple_common(Tuplesortstate *state, SortTuple *tuple)
|
||||
{
|
||||
Assert(!LEADER(state));
|
||||
|
||||
switch (state->status)
|
||||
{
|
||||
case TSS_INITIAL:
|
||||
@@ -1556,7 +1692,7 @@ puttuple_common(Tuplesortstate *state, SortTuple *tuple)
|
||||
/*
|
||||
* Nope; time to switch to tape-based operation.
|
||||
*/
|
||||
inittapes(state);
|
||||
inittapes(state, true);
|
||||
|
||||
/*
|
||||
* Dump all tuples.
|
||||
@@ -1658,8 +1794,8 @@ tuplesort_performsort(Tuplesortstate *state)
|
||||
|
||||
#ifdef TRACE_SORT
|
||||
if (trace_sort)
|
||||
elog(LOG, "performsort starting: %s",
|
||||
pg_rusage_show(&state->ru_start));
|
||||
elog(LOG, "performsort of %d starting: %s",
|
||||
state->worker, pg_rusage_show(&state->ru_start));
|
||||
#endif
|
||||
|
||||
switch (state->status)
|
||||
@@ -1668,14 +1804,39 @@ tuplesort_performsort(Tuplesortstate *state)
|
||||
|
||||
/*
|
||||
* We were able to accumulate all the tuples within the allowed
|
||||
* amount of memory. Just qsort 'em and we're done.
|
||||
* amount of memory, or leader to take over worker tapes
|
||||
*/
|
||||
tuplesort_sort_memtuples(state);
|
||||
if (SERIAL(state))
|
||||
{
|
||||
/* Just qsort 'em and we're done */
|
||||
tuplesort_sort_memtuples(state);
|
||||
state->status = TSS_SORTEDINMEM;
|
||||
}
|
||||
else if (WORKER(state))
|
||||
{
|
||||
/*
|
||||
* Parallel workers must still dump out tuples to tape. No
|
||||
* merge is required to produce single output run, though.
|
||||
*/
|
||||
inittapes(state, false);
|
||||
dumptuples(state, true);
|
||||
worker_nomergeruns(state);
|
||||
state->status = TSS_SORTEDONTAPE;
|
||||
}
|
||||
else
|
||||
{
|
||||
/*
|
||||
* Leader will take over worker tapes and merge worker runs.
|
||||
* Note that mergeruns sets the correct state->status.
|
||||
*/
|
||||
leader_takeover_tapes(state);
|
||||
mergeruns(state);
|
||||
}
|
||||
state->current = 0;
|
||||
state->eof_reached = false;
|
||||
state->markpos_block = 0L;
|
||||
state->markpos_offset = 0;
|
||||
state->markpos_eof = false;
|
||||
state->status = TSS_SORTEDINMEM;
|
||||
break;
|
||||
|
||||
case TSS_BOUNDED:
|
||||
@@ -1698,8 +1859,8 @@ tuplesort_performsort(Tuplesortstate *state)
|
||||
/*
|
||||
* Finish tape-based sort. First, flush all tuples remaining in
|
||||
* memory out to tape; then merge until we have a single remaining
|
||||
* run (or, if !randomAccess, one run per tape). Note that
|
||||
* mergeruns sets the correct state->status.
|
||||
* run (or, if !randomAccess and !WORKER(), one run per tape).
|
||||
* Note that mergeruns sets the correct state->status.
|
||||
*/
|
||||
dumptuples(state, true);
|
||||
mergeruns(state);
|
||||
@@ -1718,12 +1879,12 @@ tuplesort_performsort(Tuplesortstate *state)
|
||||
if (trace_sort)
|
||||
{
|
||||
if (state->status == TSS_FINALMERGE)
|
||||
elog(LOG, "performsort done (except %d-way final merge): %s",
|
||||
state->activeTapes,
|
||||
elog(LOG, "performsort of %d done (except %d-way final merge): %s",
|
||||
state->worker, state->activeTapes,
|
||||
pg_rusage_show(&state->ru_start));
|
||||
else
|
||||
elog(LOG, "performsort done: %s",
|
||||
pg_rusage_show(&state->ru_start));
|
||||
elog(LOG, "performsort of %d done: %s",
|
||||
state->worker, pg_rusage_show(&state->ru_start));
|
||||
}
|
||||
#endif
|
||||
|
||||
@@ -1744,6 +1905,8 @@ tuplesort_gettuple_common(Tuplesortstate *state, bool forward,
|
||||
unsigned int tuplen;
|
||||
size_t nmoved;
|
||||
|
||||
Assert(!WORKER(state));
|
||||
|
||||
switch (state->status)
|
||||
{
|
||||
case TSS_SORTEDINMEM:
|
||||
@@ -2127,6 +2290,7 @@ tuplesort_skiptuples(Tuplesortstate *state, int64 ntuples, bool forward)
|
||||
*/
|
||||
Assert(forward);
|
||||
Assert(ntuples >= 0);
|
||||
Assert(!WORKER(state));
|
||||
|
||||
switch (state->status)
|
||||
{
|
||||
@@ -2221,57 +2385,40 @@ tuplesort_merge_order(int64 allowedMem)
|
||||
/*
|
||||
* inittapes - initialize for tape sorting.
|
||||
*
|
||||
* This is called only if we have found we don't have room to sort in memory.
|
||||
* This is called only if we have found we won't sort in memory.
|
||||
*/
|
||||
static void
|
||||
inittapes(Tuplesortstate *state)
|
||||
inittapes(Tuplesortstate *state, bool mergeruns)
|
||||
{
|
||||
int maxTapes,
|
||||
j;
|
||||
int64 tapeSpace;
|
||||
|
||||
/* Compute number of tapes to use: merge order plus 1 */
|
||||
maxTapes = tuplesort_merge_order(state->allowedMem) + 1;
|
||||
Assert(!LEADER(state));
|
||||
|
||||
state->maxTapes = maxTapes;
|
||||
state->tapeRange = maxTapes - 1;
|
||||
if (mergeruns)
|
||||
{
|
||||
/* Compute number of tapes to use: merge order plus 1 */
|
||||
maxTapes = tuplesort_merge_order(state->allowedMem) + 1;
|
||||
}
|
||||
else
|
||||
{
|
||||
/* Workers can sometimes produce single run, output without merge */
|
||||
Assert(WORKER(state));
|
||||
maxTapes = MINORDER + 1;
|
||||
}
|
||||
|
||||
#ifdef TRACE_SORT
|
||||
if (trace_sort)
|
||||
elog(LOG, "switching to external sort with %d tapes: %s",
|
||||
maxTapes, pg_rusage_show(&state->ru_start));
|
||||
elog(LOG, "%d switching to external sort with %d tapes: %s",
|
||||
state->worker, maxTapes, pg_rusage_show(&state->ru_start));
|
||||
#endif
|
||||
|
||||
/*
|
||||
* Decrease availMem to reflect the space needed for tape buffers, when
|
||||
* writing the initial runs; but don't decrease it to the point that we
|
||||
* have no room for tuples. (That case is only likely to occur if sorting
|
||||
* pass-by-value Datums; in all other scenarios the memtuples[] array is
|
||||
* unlikely to occupy more than half of allowedMem. In the pass-by-value
|
||||
* case it's not important to account for tuple space, so we don't care if
|
||||
* LACKMEM becomes inaccurate.)
|
||||
*/
|
||||
tapeSpace = (int64) maxTapes * TAPE_BUFFER_OVERHEAD;
|
||||
|
||||
if (tapeSpace + GetMemoryChunkSpace(state->memtuples) < state->allowedMem)
|
||||
USEMEM(state, tapeSpace);
|
||||
|
||||
/*
|
||||
* Make sure that the temp file(s) underlying the tape set are created in
|
||||
* suitable temp tablespaces.
|
||||
*/
|
||||
PrepareTempTablespaces();
|
||||
|
||||
/*
|
||||
* Create the tape set and allocate the per-tape data arrays.
|
||||
*/
|
||||
state->tapeset = LogicalTapeSetCreate(maxTapes);
|
||||
|
||||
state->mergeactive = (bool *) palloc0(maxTapes * sizeof(bool));
|
||||
state->tp_fib = (int *) palloc0(maxTapes * sizeof(int));
|
||||
state->tp_runs = (int *) palloc0(maxTapes * sizeof(int));
|
||||
state->tp_dummy = (int *) palloc0(maxTapes * sizeof(int));
|
||||
state->tp_tapenum = (int *) palloc0(maxTapes * sizeof(int));
|
||||
/* Create the tape set and allocate the per-tape data arrays */
|
||||
inittapestate(state, maxTapes);
|
||||
state->tapeset =
|
||||
LogicalTapeSetCreate(maxTapes, NULL,
|
||||
state->shared ? &state->shared->fileset : NULL,
|
||||
state->worker);
|
||||
|
||||
state->currentRun = 0;
|
||||
|
||||
@@ -2294,6 +2441,47 @@ inittapes(Tuplesortstate *state)
|
||||
state->status = TSS_BUILDRUNS;
|
||||
}
|
||||
|
||||
/*
|
||||
* inittapestate - initialize generic tape management state
|
||||
*/
|
||||
static void
|
||||
inittapestate(Tuplesortstate *state, int maxTapes)
|
||||
{
|
||||
int64 tapeSpace;
|
||||
|
||||
/*
|
||||
* Decrease availMem to reflect the space needed for tape buffers; but
|
||||
* don't decrease it to the point that we have no room for tuples. (That
|
||||
* case is only likely to occur if sorting pass-by-value Datums; in all
|
||||
* other scenarios the memtuples[] array is unlikely to occupy more than
|
||||
* half of allowedMem. In the pass-by-value case it's not important to
|
||||
* account for tuple space, so we don't care if LACKMEM becomes
|
||||
* inaccurate.)
|
||||
*/
|
||||
tapeSpace = (int64) maxTapes * TAPE_BUFFER_OVERHEAD;
|
||||
|
||||
if (tapeSpace + GetMemoryChunkSpace(state->memtuples) < state->allowedMem)
|
||||
USEMEM(state, tapeSpace);
|
||||
|
||||
/*
|
||||
* Make sure that the temp file(s) underlying the tape set are created in
|
||||
* suitable temp tablespaces. For parallel sorts, this should have been
|
||||
* called already, but it doesn't matter if it is called a second time.
|
||||
*/
|
||||
PrepareTempTablespaces();
|
||||
|
||||
state->mergeactive = (bool *) palloc0(maxTapes * sizeof(bool));
|
||||
state->tp_fib = (int *) palloc0(maxTapes * sizeof(int));
|
||||
state->tp_runs = (int *) palloc0(maxTapes * sizeof(int));
|
||||
state->tp_dummy = (int *) palloc0(maxTapes * sizeof(int));
|
||||
state->tp_tapenum = (int *) palloc0(maxTapes * sizeof(int));
|
||||
|
||||
/* Record # of tapes allocated (for duration of sort) */
|
||||
state->maxTapes = maxTapes;
|
||||
/* Record maximum # of tapes usable as inputs when merging */
|
||||
state->tapeRange = maxTapes - 1;
|
||||
}
|
||||
|
||||
/*
|
||||
* selectnewtape -- select new tape for new initial run.
|
||||
*
|
||||
@@ -2471,8 +2659,8 @@ mergeruns(Tuplesortstate *state)
|
||||
*/
|
||||
#ifdef TRACE_SORT
|
||||
if (trace_sort)
|
||||
elog(LOG, "using " INT64_FORMAT " KB of memory for read buffers among %d input tapes",
|
||||
(state->availMem) / 1024, numInputTapes);
|
||||
elog(LOG, "%d using " INT64_FORMAT " KB of memory for read buffers among %d input tapes",
|
||||
state->worker, state->availMem / 1024, numInputTapes);
|
||||
#endif
|
||||
|
||||
state->read_buffer_size = Max(state->availMem / numInputTapes, 0);
|
||||
@@ -2490,7 +2678,7 @@ mergeruns(Tuplesortstate *state)
|
||||
* pass remains. If we don't have to produce a materialized sorted
|
||||
* tape, we can stop at this point and do the final merge on-the-fly.
|
||||
*/
|
||||
if (!state->randomAccess)
|
||||
if (!state->randomAccess && !WORKER(state))
|
||||
{
|
||||
bool allOneRun = true;
|
||||
|
||||
@@ -2575,7 +2763,10 @@ mergeruns(Tuplesortstate *state)
|
||||
* a waste of cycles anyway...
|
||||
*/
|
||||
state->result_tape = state->tp_tapenum[state->tapeRange];
|
||||
LogicalTapeFreeze(state->tapeset, state->result_tape);
|
||||
if (!WORKER(state))
|
||||
LogicalTapeFreeze(state->tapeset, state->result_tape, NULL);
|
||||
else
|
||||
worker_freeze_result_tape(state);
|
||||
state->status = TSS_SORTEDONTAPE;
|
||||
|
||||
/* Release the read buffers of all the other tapes, by rewinding them. */
|
||||
@@ -2644,8 +2835,8 @@ mergeonerun(Tuplesortstate *state)
|
||||
|
||||
#ifdef TRACE_SORT
|
||||
if (trace_sort)
|
||||
elog(LOG, "finished %d-way merge step: %s", state->activeTapes,
|
||||
pg_rusage_show(&state->ru_start));
|
||||
elog(LOG, "%d finished %d-way merge step: %s", state->worker,
|
||||
state->activeTapes, pg_rusage_show(&state->ru_start));
|
||||
#endif
|
||||
}
|
||||
|
||||
@@ -2779,8 +2970,9 @@ dumptuples(Tuplesortstate *state, bool alltuples)
|
||||
|
||||
#ifdef TRACE_SORT
|
||||
if (trace_sort)
|
||||
elog(LOG, "starting quicksort of run %d: %s",
|
||||
state->currentRun, pg_rusage_show(&state->ru_start));
|
||||
elog(LOG, "%d starting quicksort of run %d: %s",
|
||||
state->worker, state->currentRun,
|
||||
pg_rusage_show(&state->ru_start));
|
||||
#endif
|
||||
|
||||
/*
|
||||
@@ -2791,8 +2983,9 @@ dumptuples(Tuplesortstate *state, bool alltuples)
|
||||
|
||||
#ifdef TRACE_SORT
|
||||
if (trace_sort)
|
||||
elog(LOG, "finished quicksort of run %d: %s",
|
||||
state->currentRun, pg_rusage_show(&state->ru_start));
|
||||
elog(LOG, "%d finished quicksort of run %d: %s",
|
||||
state->worker, state->currentRun,
|
||||
pg_rusage_show(&state->ru_start));
|
||||
#endif
|
||||
|
||||
memtupwrite = state->memtupcount;
|
||||
@@ -2818,8 +3011,8 @@ dumptuples(Tuplesortstate *state, bool alltuples)
|
||||
|
||||
#ifdef TRACE_SORT
|
||||
if (trace_sort)
|
||||
elog(LOG, "finished writing run %d to tape %d: %s",
|
||||
state->currentRun, state->destTape,
|
||||
elog(LOG, "%d finished writing run %d to tape %d: %s",
|
||||
state->worker, state->currentRun, state->destTape,
|
||||
pg_rusage_show(&state->ru_start));
|
||||
#endif
|
||||
|
||||
@@ -3031,6 +3224,7 @@ make_bounded_heap(Tuplesortstate *state)
|
||||
Assert(state->status == TSS_INITIAL);
|
||||
Assert(state->bounded);
|
||||
Assert(tupcount >= state->bound);
|
||||
Assert(SERIAL(state));
|
||||
|
||||
/* Reverse sort direction so largest entry will be at root */
|
||||
reversedirection(state);
|
||||
@@ -3078,6 +3272,7 @@ sort_bounded_heap(Tuplesortstate *state)
|
||||
Assert(state->status == TSS_BOUNDED);
|
||||
Assert(state->bounded);
|
||||
Assert(tupcount == state->bound);
|
||||
Assert(SERIAL(state));
|
||||
|
||||
/*
|
||||
* We can unheapify in place because each delete-top call will remove the
|
||||
@@ -3112,6 +3307,8 @@ sort_bounded_heap(Tuplesortstate *state)
|
||||
static void
|
||||
tuplesort_sort_memtuples(Tuplesortstate *state)
|
||||
{
|
||||
Assert(!LEADER(state));
|
||||
|
||||
if (state->memtupcount > 1)
|
||||
{
|
||||
/* Can we use the single-key sort function? */
|
||||
@@ -4151,6 +4348,230 @@ readtup_datum(Tuplesortstate *state, SortTuple *stup,
|
||||
&tuplen, sizeof(tuplen));
|
||||
}
|
||||
|
||||
/*
|
||||
* Parallel sort routines
|
||||
*/
|
||||
|
||||
/*
|
||||
* tuplesort_estimate_shared - estimate required shared memory allocation
|
||||
*
|
||||
* nWorkers is an estimate of the number of workers (it's the number that
|
||||
* will be requested).
|
||||
*/
|
||||
Size
|
||||
tuplesort_estimate_shared(int nWorkers)
|
||||
{
|
||||
Size tapesSize;
|
||||
|
||||
Assert(nWorkers > 0);
|
||||
|
||||
/* Make sure that BufFile shared state is MAXALIGN'd */
|
||||
tapesSize = mul_size(sizeof(TapeShare), nWorkers);
|
||||
tapesSize = MAXALIGN(add_size(tapesSize, offsetof(Sharedsort, tapes)));
|
||||
|
||||
return tapesSize;
|
||||
}
|
||||
|
||||
/*
|
||||
* tuplesort_initialize_shared - initialize shared tuplesort state
|
||||
*
|
||||
* Must be called from leader process before workers are launched, to
|
||||
* establish state needed up-front for worker tuplesortstates. nWorkers
|
||||
* should match the argument passed to tuplesort_estimate_shared().
|
||||
*/
|
||||
void
|
||||
tuplesort_initialize_shared(Sharedsort *shared, int nWorkers, dsm_segment *seg)
|
||||
{
|
||||
int i;
|
||||
|
||||
Assert(nWorkers > 0);
|
||||
|
||||
SpinLockInit(&shared->mutex);
|
||||
shared->currentWorker = 0;
|
||||
shared->workersFinished = 0;
|
||||
SharedFileSetInit(&shared->fileset, seg);
|
||||
shared->nTapes = nWorkers;
|
||||
for (i = 0; i < nWorkers; i++)
|
||||
{
|
||||
shared->tapes[i].firstblocknumber = 0L;
|
||||
shared->tapes[i].buffilesize = 0;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* tuplesort_attach_shared - attach to shared tuplesort state
|
||||
*
|
||||
* Must be called by all worker processes.
|
||||
*/
|
||||
void
|
||||
tuplesort_attach_shared(Sharedsort *shared, dsm_segment *seg)
|
||||
{
|
||||
/* Attach to SharedFileSet */
|
||||
SharedFileSetAttach(&shared->fileset, seg);
|
||||
}
|
||||
|
||||
/*
|
||||
* worker_get_identifier - Assign and return ordinal identifier for worker
|
||||
*
|
||||
* The order in which these are assigned is not well defined, and should not
|
||||
* matter; worker numbers across parallel sort participants need only be
|
||||
* distinct and gapless. logtape.c requires this.
|
||||
*
|
||||
* Note that the identifiers assigned from here have no relation to
|
||||
* ParallelWorkerNumber number, to avoid making any assumption about
|
||||
* caller's requirements. However, we do follow the ParallelWorkerNumber
|
||||
* convention of representing a non-worker with worker number -1. This
|
||||
* includes the leader, as well as serial Tuplesort processes.
|
||||
*/
|
||||
static int
|
||||
worker_get_identifier(Tuplesortstate *state)
|
||||
{
|
||||
Sharedsort *shared = state->shared;
|
||||
int worker;
|
||||
|
||||
Assert(WORKER(state));
|
||||
|
||||
SpinLockAcquire(&shared->mutex);
|
||||
worker = shared->currentWorker++;
|
||||
SpinLockRelease(&shared->mutex);
|
||||
|
||||
return worker;
|
||||
}
|
||||
|
||||
/*
|
||||
* worker_freeze_result_tape - freeze worker's result tape for leader
|
||||
*
|
||||
* This is called by workers just after the result tape has been determined,
|
||||
* instead of calling LogicalTapeFreeze() directly. They do so because
|
||||
* workers require a few additional steps over similar serial
|
||||
* TSS_SORTEDONTAPE external sort cases, which also happen here. The extra
|
||||
* steps are around freeing now unneeded resources, and representing to
|
||||
* leader that worker's input run is available for its merge.
|
||||
*
|
||||
* There should only be one final output run for each worker, which consists
|
||||
* of all tuples that were originally input into worker.
|
||||
*/
|
||||
static void
|
||||
worker_freeze_result_tape(Tuplesortstate *state)
|
||||
{
|
||||
Sharedsort *shared = state->shared;
|
||||
TapeShare output;
|
||||
|
||||
Assert(WORKER(state));
|
||||
Assert(state->result_tape != -1);
|
||||
Assert(state->memtupcount == 0);
|
||||
|
||||
/*
|
||||
* Free most remaining memory, in case caller is sensitive to our holding
|
||||
* on to it. memtuples may not be a tiny merge heap at this point.
|
||||
*/
|
||||
pfree(state->memtuples);
|
||||
/* Be tidy */
|
||||
state->memtuples = NULL;
|
||||
state->memtupsize = 0;
|
||||
|
||||
/*
|
||||
* Parallel worker requires result tape metadata, which is to be stored in
|
||||
* shared memory for leader
|
||||
*/
|
||||
LogicalTapeFreeze(state->tapeset, state->result_tape, &output);
|
||||
|
||||
/* Store properties of output tape, and update finished worker count */
|
||||
SpinLockAcquire(&shared->mutex);
|
||||
shared->tapes[state->worker] = output;
|
||||
shared->workersFinished++;
|
||||
SpinLockRelease(&shared->mutex);
|
||||
}
|
||||
|
||||
/*
|
||||
* worker_nomergeruns - dump memtuples in worker, without merging
|
||||
*
|
||||
* This called as an alternative to mergeruns() with a worker when no
|
||||
* merging is required.
|
||||
*/
|
||||
static void
|
||||
worker_nomergeruns(Tuplesortstate *state)
|
||||
{
|
||||
Assert(WORKER(state));
|
||||
Assert(state->result_tape == -1);
|
||||
|
||||
state->result_tape = state->tp_tapenum[state->destTape];
|
||||
worker_freeze_result_tape(state);
|
||||
}
|
||||
|
||||
/*
|
||||
* leader_takeover_tapes - create tapeset for leader from worker tapes
|
||||
*
|
||||
* So far, leader Tuplesortstate has performed no actual sorting. By now, all
|
||||
* sorting has occurred in workers, all of which must have already returned
|
||||
* from tuplesort_performsort().
|
||||
*
|
||||
* When this returns, leader process is left in a state that is virtually
|
||||
* indistinguishable from it having generated runs as a serial external sort
|
||||
* might have.
|
||||
*/
|
||||
static void
|
||||
leader_takeover_tapes(Tuplesortstate *state)
|
||||
{
|
||||
Sharedsort *shared = state->shared;
|
||||
int nParticipants = state->nParticipants;
|
||||
int workersFinished;
|
||||
int j;
|
||||
|
||||
Assert(LEADER(state));
|
||||
Assert(nParticipants >= 1);
|
||||
|
||||
SpinLockAcquire(&shared->mutex);
|
||||
workersFinished = shared->workersFinished;
|
||||
SpinLockRelease(&shared->mutex);
|
||||
|
||||
if (nParticipants != workersFinished)
|
||||
elog(ERROR, "cannot take over tapes before all workers finish");
|
||||
|
||||
/*
|
||||
* Create the tapeset from worker tapes, including a leader-owned tape at
|
||||
* the end. Parallel workers are far more expensive than logical tapes,
|
||||
* so the number of tapes allocated here should never be excessive.
|
||||
*
|
||||
* We still have a leader tape, though it's not possible to write to it
|
||||
* due to restrictions in the shared fileset infrastructure used by
|
||||
* logtape.c. It will never be written to in practice because
|
||||
* randomAccess is disallowed for parallel sorts.
|
||||
*/
|
||||
inittapestate(state, nParticipants + 1);
|
||||
state->tapeset = LogicalTapeSetCreate(nParticipants + 1, shared->tapes,
|
||||
&shared->fileset, state->worker);
|
||||
|
||||
/* mergeruns() relies on currentRun for # of runs (in one-pass cases) */
|
||||
state->currentRun = nParticipants;
|
||||
|
||||
/*
|
||||
* Initialize variables of Algorithm D to be consistent with runs from
|
||||
* workers having been generated in the leader.
|
||||
*
|
||||
* There will always be exactly 1 run per worker, and exactly one input
|
||||
* tape per run, because workers always output exactly 1 run, even when
|
||||
* there were no input tuples for workers to sort.
|
||||
*/
|
||||
for (j = 0; j < state->maxTapes; j++)
|
||||
{
|
||||
/* One real run; no dummy runs for worker tapes */
|
||||
state->tp_fib[j] = 1;
|
||||
state->tp_runs[j] = 1;
|
||||
state->tp_dummy[j] = 0;
|
||||
state->tp_tapenum[j] = j;
|
||||
}
|
||||
/* Leader tape gets one dummy run, and no real runs */
|
||||
state->tp_fib[state->tapeRange] = 0;
|
||||
state->tp_runs[state->tapeRange] = 0;
|
||||
state->tp_dummy[state->tapeRange] = 1;
|
||||
|
||||
state->Level = 1;
|
||||
state->destTape = 0;
|
||||
|
||||
state->status = TSS_BUILDRUNS;
|
||||
}
|
||||
|
||||
/*
|
||||
* Convenience routine to free a tuple previously loaded into sort memory
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user