* 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).
+ * tuplesort.c routines across multiple workers, tapes and tape sets can
+ * be passed between processes. To do that, a SharedFileSet must be
+ * specified when the tape set is created, and the tapes must be frozen
+ * before exporting them. In the receiving process, use
+ * LogicalTapeSetImport() and LogicalTapeImport().
*
* Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*/
typedef struct LogicalTape
{
+ LogicalTapeSet *tapeSet; /* tape set this tape belongs to */
+
bool writing; /* T while in write phase */
bool frozen; /* T if blocks should not be freed when read */
- bool dirty; /* does buffer need to be written? */
+
+ long numBlocks; /* Length of the tape, in blocks */
/*
* Block numbers of the first, current, and next block of the tape.
* 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;
* 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. nHoleBlocks tracks the total number of
- * blocks that are in unused holes between worker spaces following BufFile
- * concatenation.
+ * to the underlying file yet.
*/
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[].
long *freeBlocks; /* resizable array */
int nFreeBlocks; /* # of currently free blocks */
int freeBlocksLen; /* current allocated length of freeBlocks[] */
-
- /* The array of logical tapes. */
- int nTapes; /* # of logical tapes in set */
- LogicalTape tapes[FLEXIBLE_ARRAY_MEMBER]; /* has nTapes nentries */
};
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);
/*
* 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)
{
* Returns true if anything was read, 'false' on EOF.
*/
static bool
-ltsReadFillBuffer(LogicalTapeSet *lts, LogicalTape *lt)
+ltsReadFillBuffer(LogicalTape *lt)
{
lt->pos = 0;
lt->nbytes = 0;
/* Fetch next block number */
if (datablocknum == -1L)
break; /* EOF */
- /* Apply worker offset, needed for leader tapesets */
- datablocknum += lt->offsetBlockNumber;
/* Read the block */
- ltsReadBlock(lts, datablocknum, (void *) thisbuf);
+ ltsReadBlock(lt->tapeSet, datablocknum, (void *) thisbuf);
if (!lt->frozen)
- ltsReleaseBlock(lts, datablocknum);
+ ltsReleaseBlock(lt->tapeSet, datablocknum);
lt->curBlockNumber = lt->nextBlockNumber;
lt->nbytes += TapeBlockGetNBytes(thisbuf);
}
/*
- * Claim ownership of a set of logical tapes from existing shared BufFiles.
+ * Create an empty tape set in a new temporary file.
*
- * 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.)
+ * If 'fileset' is given, a result tape can be passed to a different
+ * process after calling LogicalTapeFreeze(). This is used in parallel
+ * sorts. Each worker processes creates a tapeset in the same 'fileset',
+ * with a different 'worker' identifier. The leader process collects
+ * the result tapes from each worker process, imports them with
+ * LogicalTapeImport(), and performs the final merge.
+ *
+ * 'worker' is ignored if 'fileset' is not given. Pass -1 to be tidy.
*/
-static void
-ltsConcatWorkerTapes(LogicalTapeSet *lts, TapeShare *shared,
- SharedFileSet *fileset)
+LogicalTapeSet *
+LogicalTapeSetCreate(SharedFileSet *fileset, int worker)
{
- LogicalTape *lt = NULL;
- long tapeblocks = 0L;
- long nphysicalblocks = 0L;
- int i;
+ LogicalTapeSet *lts;
- /* Should have at least one worker tape, plus leader's tape */
- Assert(lts->nTapes >= 2);
+ /*
+ * Create top-level struct.
+ */
+ lts = (LogicalTapeSet *) palloc(sizeof(LogicalTapeSet));
+ lts->nBlocksAllocated = 0L;
+ lts->nBlocksWritten = 0L;
+ lts->forgetFreeSpace = false;
+ lts->blocksSorted = true; /* a zero-length array is sorted ... */
+ lts->freeBlocksLen = 32; /* reasonable initial guess */
+ lts->freeBlocks = (long *) palloc(lts->freeBlocksLen * sizeof(long));
+ lts->nFreeBlocks = 0;
/*
- * Build concatenated view of all BufFiles, remembering the block number
- * where each source file begins. No changes are needed for leader/last
- * tape.
+ * Create temp BufFile storage as required.
+ *
+ * Workers use a shared fileset, so that it can be passed to the leader
+ * process, whereas serial sorts use a conventional serial BufFile.
*/
- for (i = 0; i < lts->nTapes - 1; i++)
+ if (fileset)
{
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;
+ pg_itoa(worker, filename);
+ lts->pfile = BufFileCreateShared(fileset, filename);
}
+ else
+ lts->pfile = BufFileCreateTemp(false);
- /*
- * 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;
+ return lts;
}
/*
- * Create a set of logical tapes in a temporary underlying file.
+ * Claim ownership of a tape set from an existing shared BufFile.
*
- * 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.
+ * 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.)
+ *
+ * LogicalTapeImport() constructs a tapeset object, and attaches the existing
+ * tape to it. It is assumed that workers export their entire output as only
+ * one final materialized tape.
*
- * 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.
+ * NOTE: shared files are currently read-only. That could be changed if it
+ * seemed worthwhile, but for now, you cannot write to an imported tape, or
+ * you get a "Bad file descriptor" error.
*/
LogicalTapeSet *
-LogicalTapeSetCreate(int ntapes, TapeShare *shared, SharedFileSet *fileset,
- int worker)
+LogicalTapeSetImport(SharedFileSet *fileset, int worker)
{
LogicalTapeSet *lts;
- LogicalTape *lt;
- int i;
+ char filename[MAXPGPATH];
+ off_t buffilesize;
/*
- * Create top-level struct including per-tape LogicalTape structs.
+ * Create a tapeset that points to the existing file.
+ *
+ * Since we won't be writing, set 'forgetFreeSpace'.
*/
- Assert(ntapes > 0);
- lts = (LogicalTapeSet *) palloc(offsetof(LogicalTapeSet, tapes) +
- ntapes * sizeof(LogicalTape));
+ lts = (LogicalTapeSet *) palloc(sizeof(LogicalTapeSet));
lts->nBlocksAllocated = 0L;
lts->nBlocksWritten = 0L;
- lts->nHoleBlocks = 0L;
- lts->forgetFreeSpace = false;
+ lts->forgetFreeSpace = true;
lts->blocksSorted = true; /* a zero-length array is sorted ... */
- lts->freeBlocksLen = 32; /* reasonable initial guess */
- lts->freeBlocks = (long *) palloc(lts->freeBlocksLen * sizeof(long));
+ lts->freeBlocksLen = 0;
+ lts->freeBlocks = NULL;
lts->nFreeBlocks = 0;
- lts->nTapes = ntapes;
- /*
- * Initialize per-tape structs. Note we allocate the I/O buffer and the
- * first block for a tape only when it is first actually written to. This
- * avoids wasting memory space when tuplesort.c overestimates the number
- * of tapes needed.
- */
- for (i = 0; i < ntapes; i++)
- {
- lt = <s->tapes[i];
- lt->writing = true;
- lt->frozen = false;
- 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;
- }
+ pg_itoa(worker, filename);
+ lts->pfile = BufFileOpenShared(fileset, filename);
/*
- * 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.
+ * Set # of allocated blocks, as well as # blocks written, to reflect
+ * the existing file's size.
*/
- 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);
+ buffilesize = BufFileSize(lts->pfile);
+ lts->nBlocksAllocated = lts->nBlocksWritten = buffilesize / BLCKSZ;
return lts;
}
/*
* Close a logical tape set and release all resources.
+ *
+ * NOTE: This doesn't close any of the tapes! You must close them
+ * first, or you can let them be destroyed along with the memory context.
*/
void
LogicalTapeSetClose(LogicalTapeSet *lts)
{
- LogicalTape *lt;
- int i;
-
BufFileClose(lts->pfile);
- for (i = 0; i < lts->nTapes; i++)
- {
- lt = <s->tapes[i];
- if (lt->buffer)
- pfree(lt->buffer);
- }
- pfree(lts->freeBlocks);
+ if (lts->freeBlocks)
+ pfree(lts->freeBlocks);
pfree(lts);
}
+/*
+ * Create a logical tape in the given tapeset.
+ *
+ * The tape is initialized in write state.
+ */
+LogicalTape *
+LogicalTapeCreate(LogicalTapeSet *lts)
+{
+ LogicalTape *lt;
+
+ lt = palloc(sizeof(LogicalTape));
+
+ /*
+ * Initialize per-tape structs. Note we allocate the I/O buffer and the
+ * first block for a tape only when it is first actually written to.
+ */
+ lt->tapeSet = lts;
+ lt->writing = true;
+ lt->frozen = false;
+ lt->firstBlockNumber = -1L;
+ lt->curBlockNumber = -1L;
+ lt->nextBlockNumber = -1L;
+ lt->buffer = NULL;
+ lt->buffer_size = 0;
+ lt->pos = 0;
+ lt->nbytes = 0;
+ lt->numBlocks = 0;
+
+ return lt;
+}
+
+/*
+ * Register a logical tape in a logical tape set that was imported from
+ * another process.
+ *
+ * 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.)
+ *
+ * NOTE: The imported tape is read-only!
+ */
+LogicalTape *
+LogicalTapeImport(LogicalTapeSet *lts, TapeShare *shared)
+{
+ LogicalTape *lt;
+
+ /*
+ * Create a tape object, pointing to the block within the file where the
+ * tape begins.
+ */
+ lt = LogicalTapeCreate(lts);
+ lt->firstBlockNumber = shared->firstblocknumber;
+
+ /*
+ * Assume that the tape fills up the whole file. That's not necessarily
+ * quite right, but 'numBlocks' doesn't need to be accurate.
+ */
+ lt->numBlocks = lts->nBlocksAllocated;
+
+ return lt;
+}
+
+/*
+ * Close a logical tape.
+ *
+ * Note: This doesn't return any blocks to the free list! You must
+ * read the tape to the end first, to reuse the space. In current use,
+ * though, we only close tapes after fully reading them.
+ */
+void
+LogicalTapeClose(LogicalTape *lt)
+{
+ if (lt->buffer)
+ pfree(lt->buffer);
+ pfree(lt);
+}
+
/*
* Mark a logical tape set as not needing management of free space anymore.
*
lts->forgetFreeSpace = true;
}
+/*
+ * Obtain total disk space currently used by a LogicalTapeSet, in blocks.
+ */
+long
+LogicalTapeSetBlocks(LogicalTapeSet *lts)
+{
+ return lts->nBlocksAllocated;
+}
+
/*
* Write to a logical tape.
*
* There are no error returns; we ereport() on failure.
*/
void
-LogicalTapeWrite(LogicalTapeSet *lts, int tapenum,
- void *ptr, size_t size)
+LogicalTapeWrite(LogicalTape *lt, void *ptr, size_t size)
{
- LogicalTape *lt;
size_t nthistime;
- 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)
{
lt->buffer = (char *) palloc(BLCKSZ);
lt->buffer_size = BLCKSZ;
- }
- if (lt->curBlockNumber == -1)
- {
+
+ Assert(lt->curBlockNumber == -1);
Assert(lt->firstBlockNumber == -1);
Assert(lt->pos == 0);
+ Assert(lt->nbytes == 0);
- lt->curBlockNumber = ltsGetFreeBlock(lts);
+ lt->curBlockNumber = ltsGetFreeBlock(lt->tapeSet);
lt->firstBlockNumber = lt->curBlockNumber;
TapeBlockGetTrailer(lt->buffer)->prev = -1L;
/* Buffer full, dump it out */
long nextBlockNumber;
- if (!lt->dirty)
- {
- /* Hmm, went directly from reading to writing? */
- elog(ERROR, "invalid logtape state: should be dirty");
- }
-
/*
* First allocate the next block, so that we can store it in the
* 'next' pointer of this block.
*/
- nextBlockNumber = ltsGetFreeBlock(lts);
+ nextBlockNumber = ltsGetFreeBlock(lt->tapeSet);
+ lt->numBlocks++;
/* set the next-pointer and dump the current block. */
TapeBlockGetTrailer(lt->buffer)->next = nextBlockNumber;
- ltsWriteBlock(lts, lt->curBlockNumber, (void *) lt->buffer);
+ ltsWriteBlock(lt->tapeSet, lt->curBlockNumber, (void *) lt->buffer);
/* initialize the prev-pointer of the next block */
TapeBlockGetTrailer(lt->buffer)->prev = lt->curBlockNumber;
memcpy(lt->buffer + lt->pos, ptr, nthistime);
- lt->dirty = true;
lt->pos += nthistime;
if (lt->nbytes < lt->pos)
lt->nbytes = lt->pos;
* byte buffer is used.
*/
void
-LogicalTapeRewindForRead(LogicalTapeSet *lts, int tapenum, size_t buffer_size)
+LogicalTapeRewindForRead(LogicalTape *lt, size_t buffer_size)
{
- LogicalTape *lt;
-
- Assert(tapenum >= 0 && tapenum < lts->nTapes);
- lt = <s->tapes[tapenum];
-
/*
* Round and cap buffer_size if needed.
*/
buffer_size = BLCKSZ;
else
{
+ /*
+ * The buffer doesn't need to be larger than the tape.
+ */
+ if (buffer_size / BLCKSZ > lt->numBlocks)
+ buffer_size = lt->numBlocks * BLCKSZ;
+
/* need at least one block */
if (buffer_size < BLCKSZ)
buffer_size = BLCKSZ;
- /* palloc() larger than max_size is unlikely to be helpful */
- if (buffer_size > lt->max_size)
- buffer_size = lt->max_size;
+ /*
+ * palloc() larger MaxAllocSize would fail (a multi-gigabyte
+ * buffer is unlikely to be helpful, anyway)
+ */
+ if (buffer_size > MaxAllocSize)
+ buffer_size = MaxAllocSize;
/* round down to BLCKSZ boundary */
buffer_size -= buffer_size % BLCKSZ;
* Completion of a write phase. Flush last partial data block, and
* rewind for normal (destructive) read.
*/
- if (lt->dirty)
+ if (lt->curBlockNumber != -1)
{
/*
* As long as we've filled the buffer at least once, its contents
lt->buffer_size - lt->nbytes);
TapeBlockSetNBytes(lt->buffer, lt->nbytes);
- ltsWriteBlock(lts, lt->curBlockNumber, (void *) lt->buffer);
+ ltsWriteBlock(lt->tapeSet, lt->curBlockNumber, (void *) lt->buffer);
}
lt->writing = false;
}
lt->nextBlockNumber = lt->firstBlockNumber;
lt->pos = 0;
lt->nbytes = 0;
- ltsReadFillBuffer(lts, lt);
-}
-
-/*
- * Rewind logical tape and switch from reading to writing.
- *
- * NOTE: we assume the caller has read the tape to the end; otherwise
- * untouched data will not have been freed. We could add more code to free
- * any unread blocks, but in current usage of this module it'd be useless
- * code.
- */
-void
-LogicalTapeRewindForWrite(LogicalTapeSet *lts, int tapenum)
-{
- LogicalTape *lt;
-
- Assert(tapenum >= 0 && tapenum < lts->nTapes);
- lt = <s->tapes[tapenum];
-
- Assert(!lt->writing && !lt->frozen);
- lt->writing = true;
- lt->dirty = false;
- lt->firstBlockNumber = -1L;
- lt->curBlockNumber = -1L;
- lt->pos = 0;
- lt->nbytes = 0;
- if (lt->buffer)
- pfree(lt->buffer);
- lt->buffer = NULL;
- lt->buffer_size = 0;
+ ltsReadFillBuffer(lt);
}
/*
* Early EOF is indicated by return value less than #bytes requested.
*/
size_t
-LogicalTapeRead(LogicalTapeSet *lts, int tapenum,
- void *ptr, size_t size)
+LogicalTapeRead(LogicalTape *lt, void *ptr, size_t size)
{
- LogicalTape *lt;
size_t nread = 0;
size_t nthistime;
- Assert(tapenum >= 0 && tapenum < lts->nTapes);
- lt = <s->tapes[tapenum];
Assert(!lt->writing);
while (size > 0)
if (lt->pos >= lt->nbytes)
{
/* Try to load more data into buffer. */
- if (!ltsReadFillBuffer(lts, lt))
+ if (!ltsReadFillBuffer(lt))
break; /* EOF */
}
* Serial sorts should set share to NULL.
*/
void
-LogicalTapeFreeze(LogicalTapeSet *lts, int tapenum, TapeShare *share)
+LogicalTapeFreeze(LogicalTape *lt, 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
* for nondestructive read.
*/
- if (lt->dirty)
+ if (lt->curBlockNumber != -1)
{
/*
* As long as we've filled the buffer at least once, its contents are
lt->buffer_size - lt->nbytes);
TapeBlockSetNBytes(lt->buffer, lt->nbytes);
- ltsWriteBlock(lts, lt->curBlockNumber, (void *) lt->buffer);
- lt->writing = false;
+ ltsWriteBlock(lt->tapeSet, lt->curBlockNumber, (void *) lt->buffer);
}
lt->writing = false;
lt->frozen = true;
if (lt->firstBlockNumber == -1L)
lt->nextBlockNumber = -1L;
- ltsReadBlock(lts, lt->curBlockNumber, (void *) lt->buffer);
+ ltsReadBlock(lt->tapeSet, lt->curBlockNumber, (void *) lt->buffer);
if (TapeBlockIsLast(lt->buffer))
lt->nextBlockNumber = -1L;
else
/* Handle extra steps when caller is to share its tapeset */
if (share)
{
- BufFileExportShared(lts->pfile);
+ BufFileExportShared(lt->tapeSet->pfile);
share->firstblocknumber = lt->firstBlockNumber;
- share->buffilesize = BufFileSize(lts->pfile);
}
}
* that case.
*/
size_t
-LogicalTapeBackspace(LogicalTapeSet *lts, int tapenum, size_t size)
+LogicalTapeBackspace(LogicalTape *lt, size_t size)
{
- LogicalTape *lt;
size_t seekpos = 0;
- Assert(tapenum >= 0 && tapenum < lts->nTapes);
- lt = <s->tapes[tapenum];
Assert(lt->frozen);
Assert(lt->buffer_size == BLCKSZ);
return seekpos;
}
- ltsReadBlock(lts, prev, (void *) lt->buffer);
+ ltsReadBlock(lt->tapeSet, prev, (void *) lt->buffer);
if (TapeBlockGetTrailer(lt->buffer)->next != lt->curBlockNumber)
elog(ERROR, "broken tape, next of block %ld is %ld, expected %ld",
* LogicalTapeTell().
*/
void
-LogicalTapeSeek(LogicalTapeSet *lts, int tapenum,
- long blocknum, int offset)
+LogicalTapeSeek(LogicalTape *lt, long blocknum, int offset)
{
- LogicalTape *lt;
-
- Assert(tapenum >= 0 && tapenum < lts->nTapes);
- lt = <s->tapes[tapenum];
Assert(lt->frozen);
Assert(offset >= 0 && offset <= TapeBlockPayloadSize);
Assert(lt->buffer_size == BLCKSZ);
if (blocknum != lt->curBlockNumber)
{
- ltsReadBlock(lts, blocknum, (void *) lt->buffer);
+ ltsReadBlock(lt->tapeSet, blocknum, (void *) lt->buffer);
lt->curBlockNumber = blocknum;
lt->nbytes = TapeBlockPayloadSize;
lt->nextBlockNumber = TapeBlockGetTrailer(lt->buffer)->next;
* the position for a seek after freezing. Not clear if anyone needs that.
*/
void
-LogicalTapeTell(LogicalTapeSet *lts, int tapenum,
- long *blocknum, int *offset)
+LogicalTapeTell(LogicalTape *lt, long *blocknum, int *offset)
{
- LogicalTape *lt;
-
- 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);
*blocknum = lt->curBlockNumber;
*offset = lt->pos;
}
-
-/*
- * Obtain total disk space currently used by a LogicalTapeSet, in blocks.
- */
-long
-LogicalTapeSetBlocks(LogicalTapeSet *lts)
-{
- return lts->nBlocksAllocated - lts->nHoleBlocks;
-}
* algorithm.
*
* See Knuth, volume 3, for more than you want to know about the external
- * sorting algorithm. Historically, we divided the input into sorted runs
- * using replacement selection, in the form of a priority tree implemented
- * as a heap (essentially his Algorithm 5.2.3H), but now we always use
- * quicksort for run generation. We merge the runs using polyphase merge,
- * Knuth's Algorithm 5.4.2D. The logical "tapes" used by Algorithm D are
- * implemented by logtape.c, which avoids space wastage by recycling disk
- * space as soon as each block is read from its "tape".
+ * sorting algorithms. The algorithm we use is a balanced k-way merge.
+ * Before PostgreSQL 10, we used the polyphase merge algorithm (Knuth's
+ * Algorithm 5.4.2D), but with modern hardware, a straightforward
+ * balanced merge is better. Knuth is assuming that tape drives are
+ * expensive beasts, and in particular that there will always be many more
+ * runs than tape drives. The polyphase merge algorithm was good at keeping
+ * all the tape drives busy, but in our implementation a "tape drive"
+ * doesn't cost much more than a few Kb of memory buffers, so we can afford
+ * to have lots of them. In particular, if we can have as many tape drives
+ * as sorted runs, we can eliminate any repeated I/O at all. The logical
+ * "tapes" are implemented by logtape.c, which avoids space wastage by
+ * recycling disk space as soon as each block is read from its "tape".
*
* The approximate amount of memory allowed for any one sort operation
* is specified in kilobytes by the caller (most pass work_mem). Initially,
* tuples just by scanning the tuple array sequentially. If we do exceed
* workMem, we begin to emit tuples into sorted runs in temporary tapes.
* When tuples are dumped in batch after quicksorting, we begin a new run
- * with a new output tape (selected per Algorithm D). After the end of the
- * input is reached, we dump out remaining tuples in memory into a final run,
- * then merge the runs using Algorithm D.
+ * with a new output tape. After the end of the input is reached, we dump
+ * out remaining tuples in memory into a final run, then merge the runs.
*
* When merging runs, we use a heap containing just the frontmost tuple from
* each source run; we repeatedly output the smallest tuple and replace it
* accesses. The pre-reading is handled by logtape.c, we just tell it how
* much memory to use for the buffers.
*
+ * In the current code we determine the number of input tapes M on the basis
+ * of workMem: we want workMem/M to be large enough that we read a fair
+ * amount of data each time we read from a tape, so as to maintain the
+ * locality of access described above. Nonetheless, with large workMem we
+ * can have many tapes. The logical "tapes" are implemented by logtape.c,
+ * which avoids space wastage by recycling disk space as soon as each block
+ * is read from its "tape".
+ *
* When the caller requests random access to the sort result, we form
* the final sorted run on a logical tape which is then "frozen", so
* that we can access it randomly. When the caller does not need random
* on-the-fly as the caller repeatedly calls tuplesort_getXXX; this
* saves one cycle of writing all the data out to disk and reading it in.
*
- * Before Postgres 8.2, we always used a seven-tape polyphase merge, on the
- * grounds that 7 is the "sweet spot" on the tapes-to-passes curve according
- * to Knuth's figure 70 (section 5.4.2). However, Knuth is assuming that
- * tape drives are expensive beasts, and in particular that there will always
- * be many more runs than tape drives. In our implementation a "tape drive"
- * doesn't cost much more than a few Kb of memory buffers, so we can afford
- * to have lots of them. In particular, if we can have as many tape drives
- * as sorted runs, we can eliminate any repeated I/O at all. In the current
- * code we determine the number of tapes M on the basis of workMem: we want
- * workMem/M to be large enough that we read a fair amount of data each time
- * we preread from a tape, so as to maintain the locality of access described
- * 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
bool tuples; /* Can SortTuple.tuple ever be set? */
int64 availMem; /* remaining memory available, in bytes */
int64 allowedMem; /* total memory allowed, in bytes */
- int maxTapes; /* number of tapes (Knuth's T) */
- int tapeRange; /* maxTapes-1 (Knuth's P) */
+ int maxInputTapes; /* max number of input tapes */
MemoryContext sortcontext; /* memory context holding most sort data */
MemoryContext tuplecontext; /* sub-context of sortcontext for tuple data */
+
+ List *tapesets;
LogicalTapeSet *tapeset; /* logtape.c object for tapes in a temp file */
/*
* SortTuple struct!), and increase state->availMem by the amount of
* memory space thereby released.
*/
- void (*writetup) (Tuplesortstate *state, int tapenum,
+ void (*writetup) (Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
/*
* from the slab memory arena, or is palloc'd, see readtup_alloc().
*/
void (*readtup) (Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int len);
+ LogicalTape *tape, unsigned int len);
/*
* This array holds the tuples now in sort memory. If we are in state
char *slabMemoryEnd; /* end of slab memory arena */
SlabSlot *slabFreeHead; /* head of free list */
- /* Buffer size to use for reading input tapes, during merge. */
- size_t read_buffer_size;
+ /* Memory to use for input tape buffers, during merge. */
+ size_t read_buffer_mem;
/*
* When we return a tuple to the caller in tuplesort_gettuple_XXX, that
int currentRun;
/*
- * Unless otherwise noted, all pointer variables below are pointers to
- * arrays of length maxTapes, holding per-tape data.
+ * Logical tapes, for merging.
+ *
+ * The initial runs are written in the output tapes. In each merge pass,
+ * the output tapes of the previous pass become the input tapes, and
+ * new output tapes are allocated as needed. When nInputTapes == nInputRuns,
+ * there is only one merge pass left.
*/
+ LogicalTape **inputTapes;
+ int nInputTapes;
+ int nInputRuns;
- /*
- * This variable is only used during merge passes. mergeactive[i] is true
- * if we are reading an input run from (actual) tape number i and have not
- * yet exhausted that run.
- */
- bool *mergeactive; /* active input run source? */
+ LogicalTape **outputTapes;
+ int nOutputTapes;
+ int nOutputRuns;
- /*
- * Variables for Algorithm D. Note that destTape is a "logical" tape
- * number, ie, an index into the tp_xxx[] arrays. Be careful to keep
- * "logical" and "actual" tape numbers straight!
- */
- int Level; /* Knuth's l */
- int destTape; /* current output tape (Knuth's j, less 1) */
- int *tp_fib; /* Target Fibonacci run counts (A[]) */
- int *tp_runs; /* # of real runs on each tape */
- int *tp_dummy; /* # of dummy runs for each tape (D[]) */
- int *tp_tapenum; /* Actual tape numbers (TAPE[]) */
- int activeTapes; /* # of active input tapes in merge pass */
+ LogicalTape *destTape; /* current output tape */
/*
* These variables are used after completion of sorting to keep track of
* the next tuple to return. (In the tape case, the tape's current read
* position is also critical state.)
*/
- int result_tape; /* actual tape number of finished output */
+ LogicalTape *result_tape; /* actual tape of finished output */
int current; /* array index (only used if SORTEDINMEM) */
bool eof_reached; /* reached EOF (needed for cursors) */
*/
/* When using this macro, beware of double evaluation of len */
-#define LogicalTapeReadExact(tapeset, tapenum, ptr, len) \
+#define LogicalTapeReadExact(tape, ptr, len) \
do { \
- if (LogicalTapeRead(tapeset, tapenum, ptr, len) != (size_t) (len)) \
+ if (LogicalTapeRead(tape, ptr, len) != (size_t) (len)) \
elog(ERROR, "unexpected end of data"); \
} while(0)
static void mergeruns(Tuplesortstate *state);
static void mergeonerun(Tuplesortstate *state);
static void beginmerge(Tuplesortstate *state);
-static bool mergereadnext(Tuplesortstate *state, int srcTape, SortTuple *stup);
+static bool mergereadnext(Tuplesortstate *state, LogicalTape *srcTape, SortTuple *stup);
static void dumptuples(Tuplesortstate *state, bool alltuples);
static void make_bounded_heap(Tuplesortstate *state);
static void sort_bounded_heap(Tuplesortstate *state);
static void tuplesort_heap_replace_top(Tuplesortstate *state, SortTuple *tuple);
static void tuplesort_heap_delete_top(Tuplesortstate *state);
static void reversedirection(Tuplesortstate *state);
-static unsigned int getlen(Tuplesortstate *state, int tapenum, bool eofOK);
-static void markrunend(Tuplesortstate *state, int tapenum);
+static unsigned int getlen(Tuplesortstate *state, LogicalTape *tape, bool eofOK);
+static void markrunend(Tuplesortstate *state, LogicalTape *tape);
static void *readtup_alloc(Tuplesortstate *state, Size tuplen);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static void copytup_heap(Tuplesortstate *state, SortTuple *stup, void *tup);
-static void writetup_heap(Tuplesortstate *state, int tapenum,
+static void writetup_heap(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_heap(Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int len);
+ LogicalTape *tape, unsigned int len);
static int comparetup_cluster(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static void copytup_cluster(Tuplesortstate *state, SortTuple *stup, void *tup);
-static void writetup_cluster(Tuplesortstate *state, int tapenum,
+static void writetup_cluster(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_cluster(Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int len);
+ LogicalTape *tape, unsigned int len);
static int comparetup_index_btree(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static void copytup_index(Tuplesortstate *state, SortTuple *stup, void *tup);
-static void writetup_index(Tuplesortstate *state, int tapenum,
+static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int len);
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static void copytup_datum(Tuplesortstate *state, SortTuple *stup, void *tup);
-static void writetup_datum(Tuplesortstate *state, int tapenum,
+static void writetup_datum(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_datum(Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int len);
+ LogicalTape *tape, unsigned int len);
static int worker_get_identifier(Tuplesortstate *state);
static void worker_freeze_result_tape(Tuplesortstate *state);
static void worker_nomergeruns(Tuplesortstate *state);
state->sortcontext = sortcontext;
state->tuplecontext = tuplecontext;
state->tapeset = NULL;
+ state->tapesets = NIL;
state->memtupcount = 0;
state->currentRun = 0;
/*
- * maxTapes, tapeRange, and Algorithm D variables will be initialized by
- * inittapes(), if needed
+ * Tape variables (inputTapes, outputTapes, etc.) will be initialized by
+ * inittapes(), if needed.
*/
- state->result_tape = -1; /* flag that result tape has not been formed */
+ state->result_tape = NULL; /* flag that result tape has not been formed */
/*
* Initialize parallel-related state based on coordination information
{
/* context swap probably not needed, but let's be safe */
MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
+ ListCell *lc;
+ bool was_external_sort = (state->tapesets != NIL);
#ifdef TRACE_SORT
long spaceUsed;
- if (state->tapeset)
- spaceUsed = LogicalTapeSetBlocks(state->tapeset);
+ if (was_external_sort)
+ {
+ spaceUsed = 0;
+ foreach(lc, state->tapesets)
+ {
+ spaceUsed += LogicalTapeSetBlocks((LogicalTapeSet *) lfirst(lc));
+ }
+ }
else
spaceUsed = (state->allowedMem - state->availMem + 1023) / 1024;
#endif
*
* Note: want to include this in reported total cost of sort, hence need
* for two #ifdef TRACE_SORT sections.
+ *
+ * We don't bother to destroy the individual tapes here, they will go away
+ * with the sortcontext.
*/
- if (state->tapeset)
- LogicalTapeSetClose(state->tapeset);
+ foreach(lc, state->tapesets)
+ {
+ LogicalTapeSetClose((LogicalTapeSet *) lfirst(lc));
+ }
#ifdef TRACE_SORT
if (trace_sort)
{
- if (state->tapeset)
+ if (was_external_sort)
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));
state->worker, spaceUsed, pg_rusage_show(&state->ru_start));
}
- TRACE_POSTGRESQL_SORT_DONE(state->tapeset != NULL, spaceUsed);
+ TRACE_POSTGRESQL_SORT_DONE(was_external_sort, spaceUsed);
#else
/*
* If you disabled TRACE_SORT, you can still probe sort__done, but you
* ain't getting space-used stats.
*/
- TRACE_POSTGRESQL_SORT_DONE(state->tapeset != NULL, 0L);
+ TRACE_POSTGRESQL_SORT_DONE(was_external_sort != NULL, 0L);
#endif
/* Free any execution state created for CLUSTER case */
{
if (state->status == TSS_FINALMERGE)
elog(LOG, "performsort of %d done (except %d-way final merge): %s",
- state->worker, state->activeTapes,
+ state->worker, state->nInputTapes,
pg_rusage_show(&state->ru_start));
else
elog(LOG, "performsort of %d done: %s",
* end of file; back up to fetch last tuple's ending length
* word. If seek fails we must have a completely empty file.
*/
- nmoved = LogicalTapeBackspace(state->tapeset,
- state->result_tape,
+ nmoved = LogicalTapeBackspace(state->result_tape,
2 * sizeof(unsigned int));
if (nmoved == 0)
return false;
* Back up and fetch previously-returned tuple's ending length
* word. If seek fails, assume we are at start of file.
*/
- nmoved = LogicalTapeBackspace(state->tapeset,
- state->result_tape,
+ nmoved = LogicalTapeBackspace(state->result_tape,
sizeof(unsigned int));
if (nmoved == 0)
return false;
/*
* Back up to get ending length word of tuple before it.
*/
- nmoved = LogicalTapeBackspace(state->tapeset,
- state->result_tape,
+ nmoved = LogicalTapeBackspace(state->result_tape,
tuplen + 2 * sizeof(unsigned int));
if (nmoved == tuplen + sizeof(unsigned int))
{
* Note: READTUP expects we are positioned after the initial
* length word of the tuple, so back up to that point.
*/
- nmoved = LogicalTapeBackspace(state->tapeset,
- state->result_tape,
+ nmoved = LogicalTapeBackspace(state->result_tape,
tuplen);
if (nmoved != tuplen)
elog(ERROR, "bogus tuple length in backward scan");
*/
if (state->memtupcount > 0)
{
- int srcTape = state->memtuples[0].tupindex;
+ int srcTapeIndex = state->memtuples[0].tupindex;
+ LogicalTape *srcTape = state->inputTapes[srcTapeIndex];
SortTuple newtup;
*stup = state->memtuples[0];
* Remove the top node from the heap.
*/
tuplesort_heap_delete_top(state);
+ state->nInputRuns--;
/*
- * Rewind to free the read buffer. It'd go away at the
- * end of the sort anyway, but better to release the
- * memory early.
+ * Close the tape. It'd go away at the end of the sort
+ * anyway, but better to release the memory early.
*/
- LogicalTapeRewindForWrite(state->tapeset, srcTape);
+ LogicalTapeClose(srcTape);
return true;
}
- newtup.tupindex = srcTape;
+ newtup.tupindex = srcTapeIndex;
tuplesort_heap_replace_top(state, &newtup);
return true;
}
* array in this calculation, but we effectively treat that as part of the
* MERGE_BUFFER_SIZE workspace.
*/
- mOrder = (allowedMem - TAPE_BUFFER_OVERHEAD) /
- (MERGE_BUFFER_SIZE + TAPE_BUFFER_OVERHEAD);
+ mOrder = (allowedMem) /
+ (MERGE_BUFFER_SIZE + TAPE_BUFFER_OVERHEAD + TAPE_BUFFER_OVERHEAD);
/*
* Even in minimum memory, use at least a MINORDER merge. On the other
static void
inittapes(Tuplesortstate *state, bool mergeruns)
{
- int maxTapes,
- j;
+ int maxInputTapes;
Assert(!LEADER(state));
if (mergeruns)
{
- /* Compute number of tapes to use: merge order plus 1 */
- maxTapes = tuplesort_merge_order(state->allowedMem) + 1;
+ /* Compute number of input tapes to use (aka merge order) */
+ maxInputTapes = tuplesort_merge_order(state->allowedMem) + 1;
}
else
{
/* Workers can sometimes produce single run, output without merge */
Assert(WORKER(state));
- maxTapes = MINORDER + 1;
+ maxInputTapes = MINORDER;
}
#ifdef TRACE_SORT
if (trace_sort)
elog(LOG, "%d switching to external sort with %d tapes: %s",
- state->worker, maxTapes, pg_rusage_show(&state->ru_start));
+ state->worker, maxInputTapes, pg_rusage_show(&state->ru_start));
#endif
- /* Create the tape set and allocate the per-tape data arrays */
- inittapestate(state, maxTapes);
+ /*
+ * Create the tape set. It is initially empty, the tapes are created as
+ * needed.
+ */
+ inittapestate(state, maxInputTapes);
state->tapeset =
- LogicalTapeSetCreate(maxTapes, NULL,
- state->shared ? &state->shared->fileset : NULL,
+ LogicalTapeSetCreate(state->shared ? &state->shared->fileset : NULL,
state->worker);
+ state->tapesets = lappend(state->tapesets, state->tapeset);
state->currentRun = 0;
-
- /*
- * Initialize variables of Algorithm D (step D1).
- */
- for (j = 0; j < maxTapes; j++)
- {
- state->tp_fib[j] = 1;
- state->tp_runs[j] = 0;
- state->tp_dummy[j] = 1;
- state->tp_tapenum[j] = j;
- }
- state->tp_fib[state->tapeRange] = 0;
- state->tp_dummy[state->tapeRange] = 0;
-
- state->Level = 1;
- state->destTape = 0;
+ state->destTape = NULL;
state->status = TSS_BUILDRUNS;
+
+ selectnewtape(state);
}
/*
* inittapestate - initialize generic tape management state
*/
static void
-inittapestate(Tuplesortstate *state, int maxTapes)
+inittapestate(Tuplesortstate *state, int maxInputTapes)
{
int64 tapeSpace;
/*
- * Decrease availMem to reflect the space needed for tape buffers; but
+ * Decrease availMem to reflect the space needed for tape buffer of the
+ * output tape; 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
* account for tuple space, so we don't care if LACKMEM becomes
* inaccurate.)
*/
- tapeSpace = (int64) maxTapes * TAPE_BUFFER_OVERHEAD;
+ tapeSpace = (int64) maxInputTapes * TAPE_BUFFER_OVERHEAD;
if (tapeSpace + GetMemoryChunkSpace(state->memtuples) < state->allowedMem)
USEMEM(state, tapeSpace);
*/
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));
+ state->inputTapes = NULL;
+ state->nInputTapes = 0;
+ state->nInputRuns = 0;
+
+ state->outputTapes = palloc0(maxInputTapes * sizeof(LogicalTape *));
+ state->nOutputTapes = 0;
+ state->nOutputRuns = 0;
/* Record # of tapes allocated (for duration of sort) */
- state->maxTapes = maxTapes;
- /* Record maximum # of tapes usable as inputs when merging */
- state->tapeRange = maxTapes - 1;
+ state->maxInputTapes = maxInputTapes;
}
/*
- * selectnewtape -- select new tape for new initial run.
+ * selectnewtape -- select next tape to output to.
*
* This is called after finishing a run when we know another run
- * must be started. This implements steps D3, D4 of Algorithm D.
+ * must be started. This is used both when building the initial
+ * runs, and during merge passes.
*/
static void
selectnewtape(Tuplesortstate *state)
{
- int j;
- int a;
-
- /* Step D3: advance j (destTape) */
- if (state->tp_dummy[state->destTape] < state->tp_dummy[state->destTape + 1])
+ if (state->nOutputRuns < state->maxInputTapes)
{
- state->destTape++;
- return;
+ /* Create a new tape to hold the next run */
+ Assert(state->outputTapes[state->nOutputRuns] == NULL);
+ Assert(state->nOutputRuns == state->nOutputTapes);
+ state->destTape = LogicalTapeCreate(state->tapeset);
+ state->outputTapes[state->nOutputRuns] = state->destTape;
+ state->nOutputTapes++;
+ state->nOutputRuns++;
}
- if (state->tp_dummy[state->destTape] != 0)
- {
- state->destTape = 0;
- return;
- }
-
- /* Step D4: increase level */
- state->Level++;
- a = state->tp_fib[0];
- for (j = 0; j < state->tapeRange; j++)
+ else
{
- state->tp_dummy[j] = a + state->tp_fib[j + 1] - state->tp_fib[j];
- state->tp_fib[j] = a + state->tp_fib[j + 1];
+ /*
+ * We have reached the max number of tapes. Append to an existing
+ * tape.
+ */
+ state->destTape = state->outputTapes[state->nOutputRuns % state->nOutputTapes];
+ state->nOutputRuns++;
}
- state->destTape = 0;
}
/*
/*
* mergeruns -- merge all the completed initial runs.
*
- * This implements steps D5, D6 of Algorithm D. All input data has
+ * This implements the Balanced k-Way Merge Algorithm. All input data has
* already been written to initial runs on tape (see dumptuples).
*/
static void
mergeruns(Tuplesortstate *state)
{
- int tapenum,
- svTape,
- svRuns,
- svDummy;
- int numTapes;
- int numInputTapes;
+ int tapenum;
+ int64 tape_buffer_mem;
Assert(state->status == TSS_BUILDRUNS);
Assert(state->memtupcount == 0);
pfree(state->memtuples);
state->memtuples = NULL;
- /*
- * If we had fewer runs than tapes, refund the memory that we imagined we
- * would need for the tape buffers of the unused tapes.
- *
- * numTapes and numInputTapes reflect the actual number of tapes we will
- * use. Note that the output tape's tape number is maxTapes - 1, so the
- * tape numbers of the used tapes are not consecutive, and you cannot just
- * loop from 0 to numTapes to visit all used tapes!
- */
- if (state->Level == 1)
- {
- numInputTapes = state->currentRun;
- numTapes = numInputTapes + 1;
- FREEMEM(state, (state->maxTapes - numTapes) * TAPE_BUFFER_OVERHEAD);
- }
- else
- {
- numInputTapes = state->tapeRange;
- numTapes = state->maxTapes;
- }
-
/*
* Initialize the slab allocator. We need one slab slot per input tape,
* for the tuples in the heap, plus one to hold the tuple last returned
* from tuplesort_gettuple. (If we're sorting pass-by-val Datums,
* however, we don't need to do allocate anything.)
*
+ * In a multi-pass merge, we could shrink this allocation for the last
+ * merge pass, if it has fewer tapes than previous passes, but we don't
+ * bother.
+ *
* From this point on, we no longer use the USEMEM()/LACKMEM() mechanism
* to track memory usage of individual tuples.
*/
if (state->tuples)
- init_slab_allocator(state, numInputTapes + 1);
+ init_slab_allocator(state, state->nOutputTapes + 1);
else
init_slab_allocator(state, 0);
/*
* Allocate a new 'memtuples' array, for the heap. It will hold one tuple
* from each input tape.
+ *
+ * We could shrink this, too, between passes in a multi-pass merge, but
+ * we don't bother. (The initial input tapes are still in outputTapes.
+ * The number of input tapes will not increase between passes.)
*/
- state->memtupsize = numInputTapes;
- state->memtuples = (SortTuple *) palloc(numInputTapes * sizeof(SortTuple));
+ state->memtupsize = state->nOutputTapes;
+ state->memtuples = (SortTuple *) palloc(state->nOutputTapes * sizeof(SortTuple));
USEMEM(state, GetMemoryChunkSpace(state->memtuples));
/*
#ifdef TRACE_SORT
if (trace_sort)
elog(LOG, "%d using " INT64_FORMAT " KB of memory for read buffers among %d input tapes",
- state->worker, state->availMem / 1024, numInputTapes);
+ state->worker, state->availMem / 1024, state->nOutputTapes);
#endif
- state->read_buffer_size = Max(state->availMem / numInputTapes, 0);
- USEMEM(state, state->read_buffer_size * numInputTapes);
+ state->read_buffer_mem = state->availMem;
+ USEMEM(state, state->availMem);
- /* End of step D2: rewind all output tapes to prepare for merging */
- for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
- LogicalTapeRewindForRead(state->tapeset, tapenum, state->read_buffer_size);
+ /* We will use all remaining memory for read buffers */
+ tape_buffer_mem = state->availMem;
+ USEMEM(state, tape_buffer_mem);
for (;;)
{
/*
- * At this point we know that tape[T] is empty. If there's just one
- * (real or dummy) run left on each input tape, then only one merge
- * 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.
+ * On the first iteration, or if we have read all the runs from the input tapes in
+ * a multi-pass merge, it's time to start a new pass. Rewind all the output tapes,
+ * and make them inputs for the next pass.
*/
- if (!state->randomAccess && !WORKER(state))
+ if (state->nInputRuns == 0)
{
- bool allOneRun = true;
-
- Assert(state->tp_runs[state->tapeRange] == 0);
- for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
+ /* Close the old, emptied, input tapes */
+ if (state->nInputTapes > 0)
{
- if (state->tp_runs[tapenum] + state->tp_dummy[tapenum] != 1)
- {
- allOneRun = false;
- break;
- }
+ for (tapenum = 0; tapenum < state->nInputTapes; tapenum++)
+ LogicalTapeClose(state->inputTapes[tapenum]);
+ pfree(state->inputTapes);
}
- if (allOneRun)
+
+ /* Previous pass's outputs become next pass's inputs. */
+ state->inputTapes = state->outputTapes;
+ state->nInputTapes = state->nOutputTapes;
+ state->nInputRuns = state->nOutputRuns;
+
+ /*
+ * Reset output tape variables. (The actual LogicalTapes will be created
+ * as needed, we just allocate a large-enough array for them here.)
+ */
+ state->outputTapes = palloc0(state->nInputTapes * sizeof(LogicalTape *));
+ state->nOutputTapes = 0;
+ state->nOutputRuns = 0;
+
+ /* Prepare the new input tapes for merge pass. */
+ for (tapenum = 0; tapenum < state->nInputTapes; tapenum++)
+ LogicalTapeRewindForRead(state->inputTapes[tapenum],
+ state->read_buffer_mem / state->nInputTapes);
+
+ /*
+ * If there's just one run left on each input tape, then only one merge 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 && state->nInputRuns <= state->nInputTapes && !WORKER(state))
{
/* Tell logtape.c we won't be writing anymore */
- LogicalTapeSetForgetFreeSpace(state->tapeset);
+ if (state->tapeset)
+ LogicalTapeSetForgetFreeSpace(state->tapeset);
/* Initialize for the final merge pass */
beginmerge(state);
state->status = TSS_FINALMERGE;
}
}
- /* Step D5: merge runs onto tape[T] until tape[P] is empty */
- while (state->tp_runs[state->tapeRange - 1] ||
- state->tp_dummy[state->tapeRange - 1])
- {
- bool allDummy = true;
-
- for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
- {
- if (state->tp_dummy[tapenum] == 0)
- {
- allDummy = false;
- break;
- }
- }
-
- if (allDummy)
- {
- state->tp_dummy[state->tapeRange]++;
- for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
- state->tp_dummy[tapenum]--;
- }
- else
- mergeonerun(state);
- }
+ /* Select an output tape */
+ selectnewtape(state);
- /* Step D6: decrease level */
- if (--state->Level == 0)
- break;
- /* rewind output tape T to use as new input */
- LogicalTapeRewindForRead(state->tapeset, state->tp_tapenum[state->tapeRange],
- state->read_buffer_size);
- /* rewind used-up input tape P, and prepare it for write pass */
- LogicalTapeRewindForWrite(state->tapeset, state->tp_tapenum[state->tapeRange - 1]);
- state->tp_runs[state->tapeRange - 1] = 0;
+ /* Merge one run from each input tape. */
+ mergeonerun(state);
/*
- * reassign tape units per step D6; note we no longer care about A[]
+ * If the input tapes are empty, and we output only one output run,
+ * we're done. The current output tape contains the final result.
*/
- svTape = state->tp_tapenum[state->tapeRange];
- svDummy = state->tp_dummy[state->tapeRange];
- svRuns = state->tp_runs[state->tapeRange];
- for (tapenum = state->tapeRange; tapenum > 0; tapenum--)
- {
- state->tp_tapenum[tapenum] = state->tp_tapenum[tapenum - 1];
- state->tp_dummy[tapenum] = state->tp_dummy[tapenum - 1];
- state->tp_runs[tapenum] = state->tp_runs[tapenum - 1];
- }
- state->tp_tapenum[0] = svTape;
- state->tp_dummy[0] = svDummy;
- state->tp_runs[0] = svRuns;
+ if (state->nInputRuns == 0 && state->nOutputRuns <= 1)
+ break;
}
/*
- * Done. Knuth says that the result is on TAPE[1], but since we exited
- * the loop without performing the last iteration of step D6, we have not
- * rearranged the tape unit assignment, and therefore the result is on
- * TAPE[T]. We need to do it this way so that we can freeze the final
- * output tape while rewinding it. The last iteration of step D6 would be
- * a waste of cycles anyway...
+ * Done.
*/
- state->result_tape = state->tp_tapenum[state->tapeRange];
+ state->result_tape = state->outputTapes[0];
if (!WORKER(state))
- LogicalTapeFreeze(state->tapeset, state->result_tape, NULL);
+ LogicalTapeFreeze(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. */
- for (tapenum = 0; tapenum < state->maxTapes; tapenum++)
- {
- if (tapenum != state->result_tape)
- LogicalTapeRewindForWrite(state->tapeset, tapenum);
- }
+ /* Release the read buffers of all the now-empty input tapes. */
+ for (tapenum = 0; tapenum < state->nInputTapes; tapenum++)
+ LogicalTapeClose(state->inputTapes[tapenum]);
}
/*
- * Merge one run from each input tape, except ones with dummy runs.
- *
- * This is the inner loop of Algorithm D step D5. We know that the
- * output tape is TAPE[T].
+ * Merge one run from each input tape.
*/
static void
mergeonerun(Tuplesortstate *state)
{
- int destTape = state->tp_tapenum[state->tapeRange];
- int srcTape;
+ int srcTapeIndex;
+ LogicalTape *srcTape;
/*
* Start the merge by loading one tuple from each active source tape into
- * the heap. We can also decrease the input run/dummy run counts.
+ * the heap.
*/
beginmerge(state);
SortTuple stup;
/* write the tuple to destTape */
- srcTape = state->memtuples[0].tupindex;
- WRITETUP(state, destTape, &state->memtuples[0]);
+ srcTapeIndex = state->memtuples[0].tupindex;
+ srcTape = state->inputTapes[srcTapeIndex];
+ WRITETUP(state, state->destTape, &state->memtuples[0]);
/* recycle the slot of the tuple we just wrote out, for the next read */
if (state->memtuples[0].tuple)
*/
if (mergereadnext(state, srcTape, &stup))
{
- stup.tupindex = srcTape;
+ stup.tupindex = srcTapeIndex;
tuplesort_heap_replace_top(state, &stup);
}
else
+ {
tuplesort_heap_delete_top(state);
+ state->nInputRuns--;
+ }
}
/*
* When the heap empties, we're done. Write an end-of-run marker on the
- * output tape, and increment its count of real runs.
+ * output tape.
*/
- markrunend(state, destTape);
- state->tp_runs[state->tapeRange]++;
+ markrunend(state, state->destTape);
#ifdef TRACE_SORT
if (trace_sort)
elog(LOG, "%d finished %d-way merge step: %s", state->worker,
- state->activeTapes, pg_rusage_show(&state->ru_start));
+ state->nInputTapes, pg_rusage_show(&state->ru_start));
#endif
}
/*
* beginmerge - initialize for a merge pass
*
- * We decrease the counts of real and dummy runs for each tape, and mark
- * which tapes contain active input runs in mergeactive[]. Then, fill the
- * merge heap with the first tuple from each active tape.
+ * Fill the merge heap with the first tuple from each input tape.
*/
static void
beginmerge(Tuplesortstate *state)
{
int activeTapes;
- int tapenum;
- int srcTape;
+ int srcTapeIndex;
/* Heap should be empty here */
Assert(state->memtupcount == 0);
- /* Adjust run counts and mark the active tapes */
- memset(state->mergeactive, 0,
- state->maxTapes * sizeof(*state->mergeactive));
- activeTapes = 0;
- for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
- {
- if (state->tp_dummy[tapenum] > 0)
- state->tp_dummy[tapenum]--;
- else
- {
- Assert(state->tp_runs[tapenum] > 0);
- state->tp_runs[tapenum]--;
- srcTape = state->tp_tapenum[tapenum];
- state->mergeactive[srcTape] = true;
- activeTapes++;
- }
- }
- Assert(activeTapes > 0);
- state->activeTapes = activeTapes;
+ activeTapes = Min(state->nInputTapes, state->nInputRuns);
- /* Load the merge heap with the first tuple from each input tape */
- for (srcTape = 0; srcTape < state->maxTapes; srcTape++)
+ for (srcTapeIndex = 0; srcTapeIndex < activeTapes; srcTapeIndex++)
{
SortTuple tup;
- if (mergereadnext(state, srcTape, &tup))
+ if (mergereadnext(state, state->inputTapes[srcTapeIndex], &tup))
{
- tup.tupindex = srcTape;
+ tup.tupindex = srcTapeIndex;
tuplesort_heap_insert(state, &tup);
}
}
* Returns false on EOF.
*/
static bool
-mergereadnext(Tuplesortstate *state, int srcTape, SortTuple *stup)
+mergereadnext(Tuplesortstate *state, LogicalTape *srcTape, SortTuple *stup)
{
unsigned int tuplen;
- if (!state->mergeactive[srcTape])
- return false; /* tape's run is already exhausted */
-
/* read next tuple, if any */
if ((tuplen = getlen(state, srcTape, true)) == 0)
- {
- state->mergeactive[srcTape] = false;
return false;
- }
READTUP(state, stup, srcTape, tuplen);
return true;
* In general, short final runs are quite possible. Rather than allowing
* a special case where there was a superfluous selectnewtape() call (i.e.
* a call with no subsequent run actually written to destTape), we prefer
- * to write out a 0 tuple run.
- *
- * mergereadnext() is prepared for 0 tuple runs, and will reliably mark
- * the tape inactive for the merge when called from beginmerge(). This
- * case is therefore similar to the case where mergeonerun() finds a dummy
- * run for the tape, and so doesn't need to merge a run from the tape (or
- * conceptually "merges" the dummy run, if you prefer). According to
- * Knuth, Algorithm D "isn't strictly optimal" in its method of
- * distribution and dummy run assignment; this edge case seems very
- * unlikely to make that appreciably worse.
+ * to write out a 0 tuple run. In the worst case, that could add another
+ * merge pass, if that pushes us over the threshold, but it's unlikely
+ * enough to not warrant a special case. (XXX: Actually, I think some
+ * refactoring to avoid that would be in order...)
*/
Assert(state->status == TSS_BUILDRUNS);
memtupwrite = state->memtupcount;
for (i = 0; i < memtupwrite; i++)
{
- WRITETUP(state, state->tp_tapenum[state->destTape],
- &state->memtuples[i]);
+ WRITETUP(state, state->destTape, &state->memtuples[i]);
state->memtupcount--;
}
*/
MemoryContextReset(state->tuplecontext);
- markrunend(state, state->tp_tapenum[state->destTape]);
- state->tp_runs[state->destTape]++;
- state->tp_dummy[state->destTape]--; /* per Alg D step D2 */
+ markrunend(state, state->destTape);
#ifdef TRACE_SORT
if (trace_sort)
elog(LOG, "%d finished writing run %d to tape %d: %s",
- state->worker, state->currentRun, state->destTape,
+ state->worker,
+ state->currentRun, (state->currentRun - 1) % state->nOutputTapes + 1,
pg_rusage_show(&state->ru_start));
#endif
state->markpos_eof = false;
break;
case TSS_SORTEDONTAPE:
- LogicalTapeRewindForRead(state->tapeset,
- state->result_tape,
- 0);
+ LogicalTapeRewindForRead(state->result_tape, 0);
state->eof_reached = false;
state->markpos_block = 0L;
state->markpos_offset = 0;
state->markpos_eof = state->eof_reached;
break;
case TSS_SORTEDONTAPE:
- LogicalTapeTell(state->tapeset,
- state->result_tape,
+ LogicalTapeTell(state->result_tape,
&state->markpos_block,
&state->markpos_offset);
state->markpos_eof = state->eof_reached;
state->eof_reached = state->markpos_eof;
break;
case TSS_SORTEDONTAPE:
- LogicalTapeSeek(state->tapeset,
- state->result_tape,
+ LogicalTapeSeek(state->result_tape,
state->markpos_block,
state->markpos_offset);
state->eof_reached = state->markpos_eof;
* to fix. Is it worth creating an API for the memory context code to
* tell us how much is actually used in sortcontext?
*/
- if (state->tapeset)
+ if (state->tapesets)
{
+ ListCell *lc;
+
stats->spaceType = SORT_SPACE_TYPE_DISK;
- stats->spaceUsed = LogicalTapeSetBlocks(state->tapeset) * (BLCKSZ / 1024);
+ stats->spaceUsed = 0;
+ foreach(lc, state->tapesets)
+ {
+ stats->spaceUsed += LogicalTapeSetBlocks((LogicalTapeSet *) lfirst(lc)) * (BLCKSZ / 1024);
+ }
}
else
{
*/
static unsigned int
-getlen(Tuplesortstate *state, int tapenum, bool eofOK)
+getlen(Tuplesortstate *state, LogicalTape *tape, bool eofOK)
{
unsigned int len;
- if (LogicalTapeRead(state->tapeset, tapenum,
+ if (LogicalTapeRead(tape,
&len, sizeof(len)) != sizeof(len))
elog(ERROR, "unexpected end of tape");
if (len == 0 && !eofOK)
}
static void
-markrunend(Tuplesortstate *state, int tapenum)
+markrunend(Tuplesortstate *state, LogicalTape *tape)
{
unsigned int len = 0;
- LogicalTapeWrite(state->tapeset, tapenum, (void *) &len, sizeof(len));
+ LogicalTapeWrite(tape, (void *) &len, sizeof(len));
}
/*
}
static void
-writetup_heap(Tuplesortstate *state, int tapenum, SortTuple *stup)
+writetup_heap(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
{
MinimalTuple tuple = (MinimalTuple) stup->tuple;
/* total on-disk footprint: */
unsigned int tuplen = tupbodylen + sizeof(int);
- LogicalTapeWrite(state->tapeset, tapenum,
- (void *) &tuplen, sizeof(tuplen));
- LogicalTapeWrite(state->tapeset, tapenum,
- (void *) tupbody, tupbodylen);
+ LogicalTapeWrite(tape, (void *) &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, (void *) tupbody, tupbodylen);
if (state->randomAccess) /* need trailing length word? */
- LogicalTapeWrite(state->tapeset, tapenum,
- (void *) &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, (void *) &tuplen, sizeof(tuplen));
if (!state->slabAllocatorUsed)
{
static void
readtup_heap(Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int len)
+ LogicalTape *tape, unsigned int len)
{
unsigned int tupbodylen = len - sizeof(int);
unsigned int tuplen = tupbodylen + MINIMAL_TUPLE_DATA_OFFSET;
/* read in the tuple proper */
tuple->t_len = tuplen;
- LogicalTapeReadExact(state->tapeset, tapenum,
- tupbody, tupbodylen);
+ LogicalTapeReadExact(tape, tupbody, tupbodylen);
if (state->randomAccess) /* need trailing length word? */
- LogicalTapeReadExact(state->tapeset, tapenum,
- &tuplen, sizeof(tuplen));
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
stup->tuple = (void *) tuple;
/* set up first-column key value */
htup.t_len = tuple->t_len + MINIMAL_TUPLE_OFFSET;
}
static void
-writetup_cluster(Tuplesortstate *state, int tapenum, SortTuple *stup)
+writetup_cluster(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
{
HeapTuple tuple = (HeapTuple) stup->tuple;
unsigned int tuplen = tuple->t_len + sizeof(ItemPointerData) + sizeof(int);
/* We need to store t_self, but not other fields of HeapTupleData */
- LogicalTapeWrite(state->tapeset, tapenum,
- &tuplen, sizeof(tuplen));
- LogicalTapeWrite(state->tapeset, tapenum,
- &tuple->t_self, sizeof(ItemPointerData));
- LogicalTapeWrite(state->tapeset, tapenum,
- tuple->t_data, tuple->t_len);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->t_self, sizeof(ItemPointerData));
+ LogicalTapeWrite(tape, tuple->t_data, tuple->t_len);
if (state->randomAccess) /* need trailing length word? */
- LogicalTapeWrite(state->tapeset, tapenum,
- &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
if (!state->slabAllocatorUsed)
{
static void
readtup_cluster(Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int tuplen)
+ LogicalTape *tape, unsigned int tuplen)
{
unsigned int t_len = tuplen - sizeof(ItemPointerData) - sizeof(int);
HeapTuple tuple = (HeapTuple) readtup_alloc(state,
/* Reconstruct the HeapTupleData header */
tuple->t_data = (HeapTupleHeader) ((char *) tuple + HEAPTUPLESIZE);
tuple->t_len = t_len;
- LogicalTapeReadExact(state->tapeset, tapenum,
- &tuple->t_self, sizeof(ItemPointerData));
+ LogicalTapeReadExact(tape, &tuple->t_self, sizeof(ItemPointerData));
/* We don't currently bother to reconstruct t_tableOid */
tuple->t_tableOid = InvalidOid;
/* Read in the tuple body */
- LogicalTapeReadExact(state->tapeset, tapenum,
- tuple->t_data, tuple->t_len);
+ LogicalTapeReadExact(tape, tuple->t_data, tuple->t_len);
if (state->randomAccess) /* need trailing length word? */
- LogicalTapeReadExact(state->tapeset, tapenum,
- &tuplen, sizeof(tuplen));
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
stup->tuple = (void *) tuple;
/* set up first-column key value, if it's a simple column */
if (state->indexInfo->ii_IndexAttrNumbers[0] != 0)
}
static void
-writetup_index(Tuplesortstate *state, int tapenum, SortTuple *stup)
+writetup_index(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
{
IndexTuple tuple = (IndexTuple) stup->tuple;
unsigned int tuplen;
tuplen = IndexTupleSize(tuple) + sizeof(tuplen);
- LogicalTapeWrite(state->tapeset, tapenum,
- (void *) &tuplen, sizeof(tuplen));
- LogicalTapeWrite(state->tapeset, tapenum,
- (void *) tuple, IndexTupleSize(tuple));
+ LogicalTapeWrite(tape, (void *) &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, (void *) tuple, IndexTupleSize(tuple));
if (state->randomAccess) /* need trailing length word? */
- LogicalTapeWrite(state->tapeset, tapenum,
- (void *) &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, (void *) &tuplen, sizeof(tuplen));
if (!state->slabAllocatorUsed)
{
static void
readtup_index(Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int len)
+ LogicalTape *tape, unsigned int len)
{
unsigned int tuplen = len - sizeof(unsigned int);
IndexTuple tuple = (IndexTuple) readtup_alloc(state, tuplen);
- LogicalTapeReadExact(state->tapeset, tapenum,
- tuple, tuplen);
+ LogicalTapeReadExact(tape, tuple, tuplen);
if (state->randomAccess) /* need trailing length word? */
- LogicalTapeReadExact(state->tapeset, tapenum,
- &tuplen, sizeof(tuplen));
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
stup->tuple = (void *) tuple;
/* set up first-column key value */
stup->datum1 = index_getattr(tuple,
}
static void
-writetup_datum(Tuplesortstate *state, int tapenum, SortTuple *stup)
+writetup_datum(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
{
void *waddr;
unsigned int tuplen;
writtenlen = tuplen + sizeof(unsigned int);
- LogicalTapeWrite(state->tapeset, tapenum,
- (void *) &writtenlen, sizeof(writtenlen));
- LogicalTapeWrite(state->tapeset, tapenum,
- waddr, tuplen);
+ LogicalTapeWrite(tape, (void *) &writtenlen, sizeof(writtenlen));
+ LogicalTapeWrite(tape, waddr, tuplen);
if (state->randomAccess) /* need trailing length word? */
- LogicalTapeWrite(state->tapeset, tapenum,
- (void *) &writtenlen, sizeof(writtenlen));
+ LogicalTapeWrite(tape, (void *) &writtenlen, sizeof(writtenlen));
if (!state->slabAllocatorUsed && stup->tuple)
{
static void
readtup_datum(Tuplesortstate *state, SortTuple *stup,
- int tapenum, unsigned int len)
+ LogicalTape *tape, unsigned int len)
{
unsigned int tuplen = len - sizeof(unsigned int);
else if (!state->tuples)
{
Assert(tuplen == sizeof(Datum));
- LogicalTapeReadExact(state->tapeset, tapenum,
- &stup->datum1, tuplen);
+ LogicalTapeReadExact(tape, &stup->datum1, tuplen);
stup->isnull1 = false;
stup->tuple = NULL;
}
{
void *raddr = readtup_alloc(state, tuplen);
- LogicalTapeReadExact(state->tapeset, tapenum,
- raddr, tuplen);
+ LogicalTapeReadExact(tape, raddr, tuplen);
stup->datum1 = PointerGetDatum(raddr);
stup->isnull1 = false;
stup->tuple = raddr;
}
if (state->randomAccess) /* need trailing length word? */
- LogicalTapeReadExact(state->tapeset, tapenum,
- &tuplen, sizeof(tuplen));
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
}
/*
for (i = 0; i < nWorkers; i++)
{
shared->tapes[i].firstblocknumber = 0L;
- shared->tapes[i].buffilesize = 0;
}
}
TapeShare output;
Assert(WORKER(state));
- Assert(state->result_tape != -1);
+ Assert(state->result_tape != NULL);
Assert(state->memtupcount == 0);
/*
* Parallel worker requires result tape metadata, which is to be stored in
* shared memory for leader
*/
- LogicalTapeFreeze(state->tapeset, state->result_tape, &output);
+ LogicalTapeFreeze(state->result_tape, &output);
/* Store properties of output tape, and update finished worker count */
SpinLockAcquire(&shared->mutex);
worker_nomergeruns(Tuplesortstate *state)
{
Assert(WORKER(state));
- Assert(state->result_tape == -1);
+ Assert(state->result_tape == NULL);
- state->result_tape = state->tp_tapenum[state->destTape];
+ state->result_tape = state->destTape;
worker_freeze_result_tape(state);
}
/*
- * leader_takeover_tapes - create tapeset for leader from worker tapes
+ * leader_takeover_tapes - import 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
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.
* 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++)
+ inittapestate(state, nParticipants);
+
+ for (j = 0; j < nParticipants; 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;
+ LogicalTapeSet *importedTapeSet;
+
+ importedTapeSet = LogicalTapeSetImport(&shared->fileset, j);
+ state->tapesets = lappend(state->tapesets, importedTapeSet);
+ state->outputTapes[j] = LogicalTapeImport(importedTapeSet, &shared->tapes[j]);
+ state->nOutputTapes++;
+ state->nOutputRuns++;
}
- /* 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;
+ /*
+ * There is no output. 'randomAccess' is disallowed for parallel sorts, so
+ * we will perform the merge on-the-fly.
+ */
state->status = TSS_BUILDRUNS;
}