]> granicus.if.org Git - postgresql/blob - src/backend/utils/sort/tuplesort.c
Fix typo in comment
[postgresql] / src / backend / utils / sort / tuplesort.c
1 /*-------------------------------------------------------------------------
2  *
3  * tuplesort.c
4  *        Generalized tuple sorting routines.
5  *
6  * This module handles sorting of heap tuples, index tuples, or single
7  * Datums (and could easily support other kinds of sortable objects,
8  * if necessary).  It works efficiently for both small and large amounts
9  * of data.  Small amounts are sorted in-memory using qsort().  Large
10  * amounts are sorted using temporary files and a standard external sort
11  * algorithm.
12  *
13  * See Knuth, volume 3, for more than you want to know about the external
14  * sorting algorithm.  Historically, we divided the input into sorted runs
15  * using replacement selection, in the form of a priority tree implemented
16  * as a heap (essentially his Algorithm 5.2.3H), but now we only do that
17  * for the first run, and only if the run would otherwise end up being very
18  * short.  We merge the runs using polyphase merge, Knuth's Algorithm
19  * 5.4.2D.  The logical "tapes" used by Algorithm D are implemented by
20  * logtape.c, which avoids space wastage by recycling disk space as soon
21  * as each block is read from its "tape".
22  *
23  * We do not use Knuth's recommended data structure (Algorithm 5.4.1R) for
24  * the replacement selection, because it uses a fixed number of records
25  * in memory at all times.  Since we are dealing with tuples that may vary
26  * considerably in size, we want to be able to vary the number of records
27  * kept in memory to ensure full utilization of the allowed sort memory
28  * space.  So, we keep the tuples in a variable-size heap, with the next
29  * record to go out at the top of the heap.  Like Algorithm 5.4.1R, each
30  * record is stored with the run number that it must go into, and we use
31  * (run number, key) as the ordering key for the heap.  When the run number
32  * at the top of the heap changes, we know that no more records of the prior
33  * run are left in the heap.  Note that there are in practice only ever two
34  * distinct run numbers, because since PostgreSQL 9.6, we only use
35  * replacement selection to form the first run.
36  *
37  * In PostgreSQL 9.6, a heap (based on Knuth's Algorithm H, with some small
38  * customizations) is only used with the aim of producing just one run,
39  * thereby avoiding all merging.  Only the first run can use replacement
40  * selection, which is why there are now only two possible valid run
41  * numbers, and why heapification is customized to not distinguish between
42  * tuples in the second run (those will be quicksorted).  We generally
43  * prefer a simple hybrid sort-merge strategy, where runs are sorted in much
44  * the same way as the entire input of an internal sort is sorted (using
45  * qsort()).  The replacement_sort_tuples GUC controls the limited remaining
46  * use of replacement selection for the first run.
47  *
48  * There are several reasons to favor a hybrid sort-merge strategy.
49  * Maintaining a priority tree/heap has poor CPU cache characteristics.
50  * Furthermore, the growth in main memory sizes has greatly diminished the
51  * value of having runs that are larger than available memory, even in the
52  * case where there is partially sorted input and runs can be made far
53  * larger by using a heap.  In most cases, a single-pass merge step is all
54  * that is required even when runs are no larger than available memory.
55  * Avoiding multiple merge passes was traditionally considered to be the
56  * major advantage of using replacement selection.
57  *
58  * The approximate amount of memory allowed for any one sort operation
59  * is specified in kilobytes by the caller (most pass work_mem).  Initially,
60  * we absorb tuples and simply store them in an unsorted array as long as
61  * we haven't exceeded workMem.  If we reach the end of the input without
62  * exceeding workMem, we sort the array using qsort() and subsequently return
63  * tuples just by scanning the tuple array sequentially.  If we do exceed
64  * workMem, we begin to emit tuples into sorted runs in temporary tapes.
65  * When tuples are dumped in batch after quicksorting, we begin a new run
66  * with a new output tape (selected per Algorithm D).  After the end of the
67  * input is reached, we dump out remaining tuples in memory into a final run
68  * (or two, when replacement selection is still used), then merge the runs
69  * using Algorithm D.
70  *
71  * When merging runs, we use a heap containing just the frontmost tuple from
72  * each source run; we repeatedly output the smallest tuple and replace it
73  * with the next tuple from its source tape (if any).  When the heap empties,
74  * the merge is complete.  The basic merge algorithm thus needs very little
75  * memory --- only M tuples for an M-way merge, and M is constrained to a
76  * small number.  However, we can still make good use of our full workMem
77  * allocation by pre-reading additional blocks from each source tape.  Without
78  * prereading, our access pattern to the temporary file would be very erratic;
79  * on average we'd read one block from each of M source tapes during the same
80  * time that we're writing M blocks to the output tape, so there is no
81  * sequentiality of access at all, defeating the read-ahead methods used by
82  * most Unix kernels.  Worse, the output tape gets written into a very random
83  * sequence of blocks of the temp file, ensuring that things will be even
84  * worse when it comes time to read that tape.  A straightforward merge pass
85  * thus ends up doing a lot of waiting for disk seeks.  We can improve matters
86  * by prereading from each source tape sequentially, loading about workMem/M
87  * bytes from each tape in turn, and making the sequential blocks immediately
88  * available for reuse.  This approach helps to localize both read and write
89  * accesses.  The pre-reading is handled by logtape.c, we just tell it how
90  * much memory to use for the buffers.
91  *
92  * When the caller requests random access to the sort result, we form
93  * the final sorted run on a logical tape which is then "frozen", so
94  * that we can access it randomly.  When the caller does not need random
95  * access, we return from tuplesort_performsort() as soon as we are down
96  * to one run per logical tape.  The final merge is then performed
97  * on-the-fly as the caller repeatedly calls tuplesort_getXXX; this
98  * saves one cycle of writing all the data out to disk and reading it in.
99  *
100  * Before Postgres 8.2, we always used a seven-tape polyphase merge, on the
101  * grounds that 7 is the "sweet spot" on the tapes-to-passes curve according
102  * to Knuth's figure 70 (section 5.4.2).  However, Knuth is assuming that
103  * tape drives are expensive beasts, and in particular that there will always
104  * be many more runs than tape drives.  In our implementation a "tape drive"
105  * doesn't cost much more than a few Kb of memory buffers, so we can afford
106  * to have lots of them.  In particular, if we can have as many tape drives
107  * as sorted runs, we can eliminate any repeated I/O at all.  In the current
108  * code we determine the number of tapes M on the basis of workMem: we want
109  * workMem/M to be large enough that we read a fair amount of data each time
110  * we preread from a tape, so as to maintain the locality of access described
111  * above.  Nonetheless, with large workMem we can have many tapes (but not
112  * too many -- see the comments in tuplesort_merge_order).
113  *
114  *
115  * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group
116  * Portions Copyright (c) 1994, Regents of the University of California
117  *
118  * IDENTIFICATION
119  *        src/backend/utils/sort/tuplesort.c
120  *
121  *-------------------------------------------------------------------------
122  */
123
124 #include "postgres.h"
125
126 #include <limits.h>
127
128 #include "access/htup_details.h"
129 #include "access/nbtree.h"
130 #include "access/hash.h"
131 #include "catalog/index.h"
132 #include "catalog/pg_am.h"
133 #include "commands/tablespace.h"
134 #include "executor/executor.h"
135 #include "miscadmin.h"
136 #include "pg_trace.h"
137 #include "utils/datum.h"
138 #include "utils/logtape.h"
139 #include "utils/lsyscache.h"
140 #include "utils/memutils.h"
141 #include "utils/pg_rusage.h"
142 #include "utils/rel.h"
143 #include "utils/sortsupport.h"
144 #include "utils/tuplesort.h"
145
146
147 /* sort-type codes for sort__start probes */
148 #define HEAP_SORT               0
149 #define INDEX_SORT              1
150 #define DATUM_SORT              2
151 #define CLUSTER_SORT    3
152
153 /* GUC variables */
154 #ifdef TRACE_SORT
155 bool            trace_sort = false;
156 #endif
157
158 #ifdef DEBUG_BOUNDED_SORT
159 bool            optimize_bounded_sort = true;
160 #endif
161
162
163 /*
164  * The objects we actually sort are SortTuple structs.  These contain
165  * a pointer to the tuple proper (might be a MinimalTuple or IndexTuple),
166  * which is a separate palloc chunk --- we assume it is just one chunk and
167  * can be freed by a simple pfree() (except during merge, when we use a
168  * simple slab allocator).  SortTuples also contain the tuple's first key
169  * column in Datum/nullflag format, and an index integer.
170  *
171  * Storing the first key column lets us save heap_getattr or index_getattr
172  * calls during tuple comparisons.  We could extract and save all the key
173  * columns not just the first, but this would increase code complexity and
174  * overhead, and wouldn't actually save any comparison cycles in the common
175  * case where the first key determines the comparison result.  Note that
176  * for a pass-by-reference datatype, datum1 points into the "tuple" storage.
177  *
178  * There is one special case: when the sort support infrastructure provides an
179  * "abbreviated key" representation, where the key is (typically) a pass by
180  * value proxy for a pass by reference type.  In this case, the abbreviated key
181  * is stored in datum1 in place of the actual first key column.
182  *
183  * When sorting single Datums, the data value is represented directly by
184  * datum1/isnull1 for pass by value types (or null values).  If the datatype is
185  * pass-by-reference and isnull1 is false, then "tuple" points to a separately
186  * palloc'd data value, otherwise "tuple" is NULL.  The value of datum1 is then
187  * either the same pointer as "tuple", or is an abbreviated key value as
188  * described above.  Accordingly, "tuple" is always used in preference to
189  * datum1 as the authoritative value for pass-by-reference cases.
190  *
191  * While building initial runs, tupindex holds the tuple's run number.
192  * Historically, the run number could meaningfully distinguish many runs, but
193  * it now only distinguishes RUN_FIRST and HEAP_RUN_NEXT, since replacement
194  * selection is always abandoned after the first run; no other run number
195  * should be represented here.  During merge passes, we re-use it to hold the
196  * input tape number that each tuple in the heap was read from.  tupindex goes
197  * unused if the sort occurs entirely in memory.
198  */
199 typedef struct
200 {
201         void       *tuple;                      /* the tuple itself */
202         Datum           datum1;                 /* value of first key column */
203         bool            isnull1;                /* is first key column NULL? */
204         int                     tupindex;               /* see notes above */
205 } SortTuple;
206
207 /*
208  * During merge, we use a pre-allocated set of fixed-size slots to hold
209  * tuples.  To avoid palloc/pfree overhead.
210  *
211  * Merge doesn't require a lot of memory, so we can afford to waste some,
212  * by using gratuitously-sized slots.  If a tuple is larger than 1 kB, the
213  * palloc() overhead is not significant anymore.
214  *
215  * 'nextfree' is valid when this chunk is in the free list.  When in use, the
216  * slot holds a tuple.
217  */
218 #define SLAB_SLOT_SIZE 1024
219
220 typedef union SlabSlot
221 {
222         union SlabSlot *nextfree;
223         char            buffer[SLAB_SLOT_SIZE];
224 } SlabSlot;
225
226 /*
227  * Possible states of a Tuplesort object.  These denote the states that
228  * persist between calls of Tuplesort routines.
229  */
230 typedef enum
231 {
232         TSS_INITIAL,                            /* Loading tuples; still within memory limit */
233         TSS_BOUNDED,                            /* Loading tuples into bounded-size heap */
234         TSS_BUILDRUNS,                          /* Loading tuples; writing to tape */
235         TSS_SORTEDINMEM,                        /* Sort completed entirely in memory */
236         TSS_SORTEDONTAPE,                       /* Sort completed, final run is on tape */
237         TSS_FINALMERGE                          /* Performing final merge on-the-fly */
238 } TupSortStatus;
239
240 /*
241  * Parameters for calculation of number of tapes to use --- see inittapes()
242  * and tuplesort_merge_order().
243  *
244  * In this calculation we assume that each tape will cost us about 1 blocks
245  * worth of buffer space.  This ignores the overhead of all the other data
246  * structures needed for each tape, but it's probably close enough.
247  *
248  * MERGE_BUFFER_SIZE is how much data we'd like to read from each input
249  * tape during a preread cycle (see discussion at top of file).
250  */
251 #define MINORDER                6               /* minimum merge order */
252 #define MAXORDER                500             /* maximum merge order */
253 #define TAPE_BUFFER_OVERHEAD            BLCKSZ
254 #define MERGE_BUFFER_SIZE                       (BLCKSZ * 32)
255
256  /*
257   * Run numbers, used during external sort operations.
258   *
259   * HEAP_RUN_NEXT is only used for SortTuple.tupindex, never state.currentRun.
260   */
261 #define RUN_FIRST               0
262 #define HEAP_RUN_NEXT   INT_MAX
263 #define RUN_SECOND              1
264
265 typedef int (*SortTupleComparator) (const SortTuple *a, const SortTuple *b,
266                                                                         Tuplesortstate *state);
267
268 /*
269  * Private state of a Tuplesort operation.
270  */
271 struct Tuplesortstate
272 {
273         TupSortStatus status;           /* enumerated value as shown above */
274         int                     nKeys;                  /* number of columns in sort key */
275         bool            randomAccess;   /* did caller request random access? */
276         bool            bounded;                /* did caller specify a maximum number of
277                                                                  * tuples to return? */
278         bool            boundUsed;              /* true if we made use of a bounded heap */
279         int                     bound;                  /* if bounded, the maximum number of tuples */
280         bool            tuples;                 /* Can SortTuple.tuple ever be set? */
281         int64           availMem;               /* remaining memory available, in bytes */
282         int64           allowedMem;             /* total memory allowed, in bytes */
283         int                     maxTapes;               /* number of tapes (Knuth's T) */
284         int                     tapeRange;              /* maxTapes-1 (Knuth's P) */
285         MemoryContext sortcontext;      /* memory context holding most sort data */
286         MemoryContext tuplecontext; /* sub-context of sortcontext for tuple data */
287         LogicalTapeSet *tapeset;        /* logtape.c object for tapes in a temp file */
288
289         /*
290          * These function pointers decouple the routines that must know what kind
291          * of tuple we are sorting from the routines that don't need to know it.
292          * They are set up by the tuplesort_begin_xxx routines.
293          *
294          * Function to compare two tuples; result is per qsort() convention, ie:
295          * <0, 0, >0 according as a<b, a=b, a>b.  The API must match
296          * qsort_arg_comparator.
297          */
298         SortTupleComparator comparetup;
299
300         /*
301          * Function to copy a supplied input tuple into palloc'd space and set up
302          * its SortTuple representation (ie, set tuple/datum1/isnull1).  Also,
303          * state->availMem must be decreased by the amount of space used for the
304          * tuple copy (note the SortTuple struct itself is not counted).
305          */
306         void            (*copytup) (Tuplesortstate *state, SortTuple *stup, void *tup);
307
308         /*
309          * Function to write a stored tuple onto tape.  The representation of the
310          * tuple on tape need not be the same as it is in memory; requirements on
311          * the tape representation are given below.  Unless the slab allocator is
312          * used, after writing the tuple, pfree() the out-of-line data (not the
313          * SortTuple struct!), and increase state->availMem by the amount of
314          * memory space thereby released.
315          */
316         void            (*writetup) (Tuplesortstate *state, int tapenum,
317                                                          SortTuple *stup);
318
319         /*
320          * Function to read a stored tuple from tape back into memory. 'len' is
321          * the already-read length of the stored tuple.  The tuple is allocated
322          * from the slab memory arena, or is palloc'd, see readtup_alloc().
323          */
324         void            (*readtup) (Tuplesortstate *state, SortTuple *stup,
325                                                         int tapenum, unsigned int len);
326
327         /*
328          * This array holds the tuples now in sort memory.  If we are in state
329          * INITIAL, the tuples are in no particular order; if we are in state
330          * SORTEDINMEM, the tuples are in final sorted order; in states BUILDRUNS
331          * and FINALMERGE, the tuples are organized in "heap" order per Algorithm
332          * H.  In state SORTEDONTAPE, the array is not used.
333          */
334         SortTuple  *memtuples;          /* array of SortTuple structs */
335         int                     memtupcount;    /* number of tuples currently present */
336         int                     memtupsize;             /* allocated length of memtuples array */
337         bool            growmemtuples;  /* memtuples' growth still underway? */
338
339         /*
340          * Memory for tuples is sometimes allocated using a simple slab allocator,
341          * rather than with palloc().  Currently, we switch to slab allocation
342          * when we start merging.  Merging only needs to keep a small, fixed
343          * number of tuples in memory at any time, so we can avoid the
344          * palloc/pfree overhead by recycling a fixed number of fixed-size slots
345          * to hold the tuples.
346          *
347          * For the slab, we use one large allocation, divided into SLAB_SLOT_SIZE
348          * slots.  The allocation is sized to have one slot per tape, plus one
349          * additional slot.  We need that many slots to hold all the tuples kept
350          * in the heap during merge, plus the one we have last returned from the
351          * sort, with tuplesort_gettuple.
352          *
353          * Initially, all the slots are kept in a linked list of free slots.  When
354          * a tuple is read from a tape, it is put to the next available slot, if
355          * it fits.  If the tuple is larger than SLAB_SLOT_SIZE, it is palloc'd
356          * instead.
357          *
358          * When we're done processing a tuple, we return the slot back to the free
359          * list, or pfree() if it was palloc'd.  We know that a tuple was
360          * allocated from the slab, if its pointer value is between
361          * slabMemoryBegin and -End.
362          *
363          * When the slab allocator is used, the USEMEM/LACKMEM mechanism of
364          * tracking memory usage is not used.
365          */
366         bool            slabAllocatorUsed;
367
368         char       *slabMemoryBegin;    /* beginning of slab memory arena */
369         char       *slabMemoryEnd;      /* end of slab memory arena */
370         SlabSlot   *slabFreeHead;       /* head of free list */
371
372         /* Buffer size to use for reading input tapes, during merge. */
373         size_t          read_buffer_size;
374
375         /*
376          * When we return a tuple to the caller in tuplesort_gettuple_XXX, that
377          * came from a tape (that is, in TSS_SORTEDONTAPE or TSS_FINALMERGE
378          * modes), we remember the tuple in 'lastReturnedTuple', so that we can
379          * recycle the memory on next gettuple call.
380          */
381         void       *lastReturnedTuple;
382
383         /*
384          * While building initial runs, this indicates if the replacement
385          * selection strategy is in use.  When it isn't, then a simple hybrid
386          * sort-merge strategy is in use instead (runs are quicksorted).
387          */
388         bool            replaceActive;
389
390         /*
391          * While building initial runs, this is the current output run number
392          * (starting at RUN_FIRST).  Afterwards, it is the number of initial runs
393          * we made.
394          */
395         int                     currentRun;
396
397         /*
398          * Unless otherwise noted, all pointer variables below are pointers to
399          * arrays of length maxTapes, holding per-tape data.
400          */
401
402         /*
403          * This variable is only used during merge passes.  mergeactive[i] is true
404          * if we are reading an input run from (actual) tape number i and have not
405          * yet exhausted that run.
406          */
407         bool       *mergeactive;        /* active input run source? */
408
409         /*
410          * Variables for Algorithm D.  Note that destTape is a "logical" tape
411          * number, ie, an index into the tp_xxx[] arrays.  Be careful to keep
412          * "logical" and "actual" tape numbers straight!
413          */
414         int                     Level;                  /* Knuth's l */
415         int                     destTape;               /* current output tape (Knuth's j, less 1) */
416         int                *tp_fib;                     /* Target Fibonacci run counts (A[]) */
417         int                *tp_runs;            /* # of real runs on each tape */
418         int                *tp_dummy;           /* # of dummy runs for each tape (D[]) */
419         int                *tp_tapenum;         /* Actual tape numbers (TAPE[]) */
420         int                     activeTapes;    /* # of active input tapes in merge pass */
421
422         /*
423          * These variables are used after completion of sorting to keep track of
424          * the next tuple to return.  (In the tape case, the tape's current read
425          * position is also critical state.)
426          */
427         int                     result_tape;    /* actual tape number of finished output */
428         int                     current;                /* array index (only used if SORTEDINMEM) */
429         bool            eof_reached;    /* reached EOF (needed for cursors) */
430
431         /* markpos_xxx holds marked position for mark and restore */
432         long            markpos_block;  /* tape block# (only used if SORTEDONTAPE) */
433         int                     markpos_offset; /* saved "current", or offset in tape block */
434         bool            markpos_eof;    /* saved "eof_reached" */
435
436         /*
437          * The sortKeys variable is used by every case other than the hash index
438          * case; it is set by tuplesort_begin_xxx.  tupDesc is only used by the
439          * MinimalTuple and CLUSTER routines, though.
440          */
441         TupleDesc       tupDesc;
442         SortSupport sortKeys;           /* array of length nKeys */
443
444         /*
445          * This variable is shared by the single-key MinimalTuple case and the
446          * Datum case (which both use qsort_ssup()).  Otherwise it's NULL.
447          */
448         SortSupport onlyKey;
449
450         /*
451          * Additional state for managing "abbreviated key" sortsupport routines
452          * (which currently may be used by all cases except the hash index case).
453          * Tracks the intervals at which the optimization's effectiveness is
454          * tested.
455          */
456         int64           abbrevNext;             /* Tuple # at which to next check
457                                                                  * applicability */
458
459         /*
460          * These variables are specific to the CLUSTER case; they are set by
461          * tuplesort_begin_cluster.
462          */
463         IndexInfo  *indexInfo;          /* info about index being used for reference */
464         EState     *estate;                     /* for evaluating index expressions */
465
466         /*
467          * These variables are specific to the IndexTuple case; they are set by
468          * tuplesort_begin_index_xxx and used only by the IndexTuple routines.
469          */
470         Relation        heapRel;                /* table the index is being built on */
471         Relation        indexRel;               /* index being built */
472
473         /* These are specific to the index_btree subcase: */
474         bool            enforceUnique;  /* complain if we find duplicate tuples */
475
476         /* These are specific to the index_hash subcase: */
477         uint32          high_mask;              /* masks for sortable part of hash code */
478         uint32          low_mask;
479         uint32          max_buckets;
480
481         /*
482          * These variables are specific to the Datum case; they are set by
483          * tuplesort_begin_datum and used only by the DatumTuple routines.
484          */
485         Oid                     datumType;
486         /* we need typelen in order to know how to copy the Datums. */
487         int                     datumTypeLen;
488
489         /*
490          * Resource snapshot for time of sort start.
491          */
492 #ifdef TRACE_SORT
493         PGRUsage        ru_start;
494 #endif
495 };
496
497 /*
498  * Is the given tuple allocated from the slab memory arena?
499  */
500 #define IS_SLAB_SLOT(state, tuple) \
501         ((char *) (tuple) >= (state)->slabMemoryBegin && \
502          (char *) (tuple) < (state)->slabMemoryEnd)
503
504 /*
505  * Return the given tuple to the slab memory free list, or free it
506  * if it was palloc'd.
507  */
508 #define RELEASE_SLAB_SLOT(state, tuple) \
509         do { \
510                 SlabSlot *buf = (SlabSlot *) tuple; \
511                 \
512                 if (IS_SLAB_SLOT((state), buf)) \
513                 { \
514                         buf->nextfree = (state)->slabFreeHead; \
515                         (state)->slabFreeHead = buf; \
516                 } else \
517                         pfree(buf); \
518         } while(0)
519
520 #define COMPARETUP(state,a,b)   ((*(state)->comparetup) (a, b, state))
521 #define COPYTUP(state,stup,tup) ((*(state)->copytup) (state, stup, tup))
522 #define WRITETUP(state,tape,stup)       ((*(state)->writetup) (state, tape, stup))
523 #define READTUP(state,stup,tape,len) ((*(state)->readtup) (state, stup, tape, len))
524 #define LACKMEM(state)          ((state)->availMem < 0 && !(state)->slabAllocatorUsed)
525 #define USEMEM(state,amt)       ((state)->availMem -= (amt))
526 #define FREEMEM(state,amt)      ((state)->availMem += (amt))
527
528 /*
529  * NOTES about on-tape representation of tuples:
530  *
531  * We require the first "unsigned int" of a stored tuple to be the total size
532  * on-tape of the tuple, including itself (so it is never zero; an all-zero
533  * unsigned int is used to delimit runs).  The remainder of the stored tuple
534  * may or may not match the in-memory representation of the tuple ---
535  * any conversion needed is the job of the writetup and readtup routines.
536  *
537  * If state->randomAccess is true, then the stored representation of the
538  * tuple must be followed by another "unsigned int" that is a copy of the
539  * length --- so the total tape space used is actually sizeof(unsigned int)
540  * more than the stored length value.  This allows read-backwards.  When
541  * randomAccess is not true, the write/read routines may omit the extra
542  * length word.
543  *
544  * writetup is expected to write both length words as well as the tuple
545  * data.  When readtup is called, the tape is positioned just after the
546  * front length word; readtup must read the tuple data and advance past
547  * the back length word (if present).
548  *
549  * The write/read routines can make use of the tuple description data
550  * stored in the Tuplesortstate record, if needed.  They are also expected
551  * to adjust state->availMem by the amount of memory space (not tape space!)
552  * released or consumed.  There is no error return from either writetup
553  * or readtup; they should ereport() on failure.
554  *
555  *
556  * NOTES about memory consumption calculations:
557  *
558  * We count space allocated for tuples against the workMem limit, plus
559  * the space used by the variable-size memtuples array.  Fixed-size space
560  * is not counted; it's small enough to not be interesting.
561  *
562  * Note that we count actual space used (as shown by GetMemoryChunkSpace)
563  * rather than the originally-requested size.  This is important since
564  * palloc can add substantial overhead.  It's not a complete answer since
565  * we won't count any wasted space in palloc allocation blocks, but it's
566  * a lot better than what we were doing before 7.3.  As of 9.6, a
567  * separate memory context is used for caller passed tuples.  Resetting
568  * it at certain key increments significantly ameliorates fragmentation.
569  * Note that this places a responsibility on readtup and copytup routines
570  * to use the right memory context for these tuples (and to not use the
571  * reset context for anything whose lifetime needs to span multiple
572  * external sort runs).
573  */
574
575 /* When using this macro, beware of double evaluation of len */
576 #define LogicalTapeReadExact(tapeset, tapenum, ptr, len) \
577         do { \
578                 if (LogicalTapeRead(tapeset, tapenum, ptr, len) != (size_t) (len)) \
579                         elog(ERROR, "unexpected end of data"); \
580         } while(0)
581
582
583 static Tuplesortstate *tuplesort_begin_common(int workMem, bool randomAccess);
584 static void puttuple_common(Tuplesortstate *state, SortTuple *tuple);
585 static bool consider_abort_common(Tuplesortstate *state);
586 static bool useselection(Tuplesortstate *state);
587 static void inittapes(Tuplesortstate *state);
588 static void selectnewtape(Tuplesortstate *state);
589 static void init_slab_allocator(Tuplesortstate *state, int numSlots);
590 static void mergeruns(Tuplesortstate *state);
591 static void mergeonerun(Tuplesortstate *state);
592 static void beginmerge(Tuplesortstate *state);
593 static bool mergereadnext(Tuplesortstate *state, int srcTape, SortTuple *stup);
594 static void dumptuples(Tuplesortstate *state, bool alltuples);
595 static void dumpbatch(Tuplesortstate *state, bool alltuples);
596 static void make_bounded_heap(Tuplesortstate *state);
597 static void sort_bounded_heap(Tuplesortstate *state);
598 static void tuplesort_sort_memtuples(Tuplesortstate *state);
599 static void tuplesort_heap_insert(Tuplesortstate *state, SortTuple *tuple,
600                                           bool checkIndex);
601 static void tuplesort_heap_replace_top(Tuplesortstate *state, SortTuple *tuple,
602                                                    bool checkIndex);
603 static void tuplesort_heap_delete_top(Tuplesortstate *state, bool checkIndex);
604 static void reversedirection(Tuplesortstate *state);
605 static unsigned int getlen(Tuplesortstate *state, int tapenum, bool eofOK);
606 static void markrunend(Tuplesortstate *state, int tapenum);
607 static void *readtup_alloc(Tuplesortstate *state, Size tuplen);
608 static int comparetup_heap(const SortTuple *a, const SortTuple *b,
609                                 Tuplesortstate *state);
610 static void copytup_heap(Tuplesortstate *state, SortTuple *stup, void *tup);
611 static void writetup_heap(Tuplesortstate *state, int tapenum,
612                           SortTuple *stup);
613 static void readtup_heap(Tuplesortstate *state, SortTuple *stup,
614                          int tapenum, unsigned int len);
615 static int comparetup_cluster(const SortTuple *a, const SortTuple *b,
616                                    Tuplesortstate *state);
617 static void copytup_cluster(Tuplesortstate *state, SortTuple *stup, void *tup);
618 static void writetup_cluster(Tuplesortstate *state, int tapenum,
619                                  SortTuple *stup);
620 static void readtup_cluster(Tuplesortstate *state, SortTuple *stup,
621                                 int tapenum, unsigned int len);
622 static int comparetup_index_btree(const SortTuple *a, const SortTuple *b,
623                                            Tuplesortstate *state);
624 static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
625                                           Tuplesortstate *state);
626 static void copytup_index(Tuplesortstate *state, SortTuple *stup, void *tup);
627 static void writetup_index(Tuplesortstate *state, int tapenum,
628                            SortTuple *stup);
629 static void readtup_index(Tuplesortstate *state, SortTuple *stup,
630                           int tapenum, unsigned int len);
631 static int comparetup_datum(const SortTuple *a, const SortTuple *b,
632                                  Tuplesortstate *state);
633 static void copytup_datum(Tuplesortstate *state, SortTuple *stup, void *tup);
634 static void writetup_datum(Tuplesortstate *state, int tapenum,
635                            SortTuple *stup);
636 static void readtup_datum(Tuplesortstate *state, SortTuple *stup,
637                           int tapenum, unsigned int len);
638 static void free_sort_tuple(Tuplesortstate *state, SortTuple *stup);
639
640 /*
641  * Special versions of qsort just for SortTuple objects.  qsort_tuple() sorts
642  * any variant of SortTuples, using the appropriate comparetup function.
643  * qsort_ssup() is specialized for the case where the comparetup function
644  * reduces to ApplySortComparator(), that is single-key MinimalTuple sorts
645  * and Datum sorts.
646  */
647 #include "qsort_tuple.c"
648
649
650 /*
651  *              tuplesort_begin_xxx
652  *
653  * Initialize for a tuple sort operation.
654  *
655  * After calling tuplesort_begin, the caller should call tuplesort_putXXX
656  * zero or more times, then call tuplesort_performsort when all the tuples
657  * have been supplied.  After performsort, retrieve the tuples in sorted
658  * order by calling tuplesort_getXXX until it returns false/NULL.  (If random
659  * access was requested, rescan, markpos, and restorepos can also be called.)
660  * Call tuplesort_end to terminate the operation and release memory/disk space.
661  *
662  * Each variant of tuplesort_begin has a workMem parameter specifying the
663  * maximum number of kilobytes of RAM to use before spilling data to disk.
664  * (The normal value of this parameter is work_mem, but some callers use
665  * other values.)  Each variant also has a randomAccess parameter specifying
666  * whether the caller needs non-sequential access to the sort result.
667  */
668
669 static Tuplesortstate *
670 tuplesort_begin_common(int workMem, bool randomAccess)
671 {
672         Tuplesortstate *state;
673         MemoryContext sortcontext;
674         MemoryContext tuplecontext;
675         MemoryContext oldcontext;
676
677         /*
678          * Create a working memory context for this sort operation. All data
679          * needed by the sort will live inside this context.
680          */
681         sortcontext = AllocSetContextCreate(CurrentMemoryContext,
682                                                                                 "TupleSort main",
683                                                                                 ALLOCSET_DEFAULT_SIZES);
684
685         /*
686          * Caller tuple (e.g. IndexTuple) memory context.
687          *
688          * A dedicated child context used exclusively for caller passed tuples
689          * eases memory management.  Resetting at key points reduces
690          * fragmentation. Note that the memtuples array of SortTuples is allocated
691          * in the parent context, not this context, because there is no need to
692          * free memtuples early.
693          */
694         tuplecontext = AllocSetContextCreate(sortcontext,
695                                                                                  "Caller tuples",
696                                                                                  ALLOCSET_DEFAULT_SIZES);
697
698         /*
699          * Make the Tuplesortstate within the per-sort context.  This way, we
700          * don't need a separate pfree() operation for it at shutdown.
701          */
702         oldcontext = MemoryContextSwitchTo(sortcontext);
703
704         state = (Tuplesortstate *) palloc0(sizeof(Tuplesortstate));
705
706 #ifdef TRACE_SORT
707         if (trace_sort)
708                 pg_rusage_init(&state->ru_start);
709 #endif
710
711         state->status = TSS_INITIAL;
712         state->randomAccess = randomAccess;
713         state->bounded = false;
714         state->tuples = true;
715         state->boundUsed = false;
716         state->allowedMem = workMem * (int64) 1024;
717         state->availMem = state->allowedMem;
718         state->sortcontext = sortcontext;
719         state->tuplecontext = tuplecontext;
720         state->tapeset = NULL;
721
722         state->memtupcount = 0;
723
724         /*
725          * Initial size of array must be more than ALLOCSET_SEPARATE_THRESHOLD;
726          * see comments in grow_memtuples().
727          */
728         state->memtupsize = Max(1024,
729                                                         ALLOCSET_SEPARATE_THRESHOLD / sizeof(SortTuple) + 1);
730
731         state->growmemtuples = true;
732         state->slabAllocatorUsed = false;
733         state->memtuples = (SortTuple *) palloc(state->memtupsize * sizeof(SortTuple));
734
735         USEMEM(state, GetMemoryChunkSpace(state->memtuples));
736
737         /* workMem must be large enough for the minimal memtuples array */
738         if (LACKMEM(state))
739                 elog(ERROR, "insufficient memory allowed for sort");
740
741         state->currentRun = RUN_FIRST;
742
743         /*
744          * maxTapes, tapeRange, and Algorithm D variables will be initialized by
745          * inittapes(), if needed
746          */
747
748         state->result_tape = -1;        /* flag that result tape has not been formed */
749
750         MemoryContextSwitchTo(oldcontext);
751
752         return state;
753 }
754
755 Tuplesortstate *
756 tuplesort_begin_heap(TupleDesc tupDesc,
757                                          int nkeys, AttrNumber *attNums,
758                                          Oid *sortOperators, Oid *sortCollations,
759                                          bool *nullsFirstFlags,
760                                          int workMem, bool randomAccess)
761 {
762         Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
763         MemoryContext oldcontext;
764         int                     i;
765
766         oldcontext = MemoryContextSwitchTo(state->sortcontext);
767
768         AssertArg(nkeys > 0);
769
770 #ifdef TRACE_SORT
771         if (trace_sort)
772                 elog(LOG,
773                          "begin tuple sort: nkeys = %d, workMem = %d, randomAccess = %c",
774                          nkeys, workMem, randomAccess ? 't' : 'f');
775 #endif
776
777         state->nKeys = nkeys;
778
779         TRACE_POSTGRESQL_SORT_START(HEAP_SORT,
780                                                                 false,  /* no unique check */
781                                                                 nkeys,
782                                                                 workMem,
783                                                                 randomAccess);
784
785         state->comparetup = comparetup_heap;
786         state->copytup = copytup_heap;
787         state->writetup = writetup_heap;
788         state->readtup = readtup_heap;
789
790         state->tupDesc = tupDesc;       /* assume we need not copy tupDesc */
791         state->abbrevNext = 10;
792
793         /* Prepare SortSupport data for each column */
794         state->sortKeys = (SortSupport) palloc0(nkeys * sizeof(SortSupportData));
795
796         for (i = 0; i < nkeys; i++)
797         {
798                 SortSupport sortKey = state->sortKeys + i;
799
800                 AssertArg(attNums[i] != 0);
801                 AssertArg(sortOperators[i] != 0);
802
803                 sortKey->ssup_cxt = CurrentMemoryContext;
804                 sortKey->ssup_collation = sortCollations[i];
805                 sortKey->ssup_nulls_first = nullsFirstFlags[i];
806                 sortKey->ssup_attno = attNums[i];
807                 /* Convey if abbreviation optimization is applicable in principle */
808                 sortKey->abbreviate = (i == 0);
809
810                 PrepareSortSupportFromOrderingOp(sortOperators[i], sortKey);
811         }
812
813         /*
814          * The "onlyKey" optimization cannot be used with abbreviated keys, since
815          * tie-breaker comparisons may be required.  Typically, the optimization
816          * is only of value to pass-by-value types anyway, whereas abbreviated
817          * keys are typically only of value to pass-by-reference types.
818          */
819         if (nkeys == 1 && !state->sortKeys->abbrev_converter)
820                 state->onlyKey = state->sortKeys;
821
822         MemoryContextSwitchTo(oldcontext);
823
824         return state;
825 }
826
827 Tuplesortstate *
828 tuplesort_begin_cluster(TupleDesc tupDesc,
829                                                 Relation indexRel,
830                                                 int workMem, bool randomAccess)
831 {
832         Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
833         ScanKey         indexScanKey;
834         MemoryContext oldcontext;
835         int                     i;
836
837         Assert(indexRel->rd_rel->relam == BTREE_AM_OID);
838
839         oldcontext = MemoryContextSwitchTo(state->sortcontext);
840
841 #ifdef TRACE_SORT
842         if (trace_sort)
843                 elog(LOG,
844                          "begin tuple sort: nkeys = %d, workMem = %d, randomAccess = %c",
845                          RelationGetNumberOfAttributes(indexRel),
846                          workMem, randomAccess ? 't' : 'f');
847 #endif
848
849         state->nKeys = RelationGetNumberOfAttributes(indexRel);
850
851         TRACE_POSTGRESQL_SORT_START(CLUSTER_SORT,
852                                                                 false,  /* no unique check */
853                                                                 state->nKeys,
854                                                                 workMem,
855                                                                 randomAccess);
856
857         state->comparetup = comparetup_cluster;
858         state->copytup = copytup_cluster;
859         state->writetup = writetup_cluster;
860         state->readtup = readtup_cluster;
861         state->abbrevNext = 10;
862
863         state->indexInfo = BuildIndexInfo(indexRel);
864
865         state->tupDesc = tupDesc;       /* assume we need not copy tupDesc */
866
867         indexScanKey = _bt_mkscankey_nodata(indexRel);
868
869         if (state->indexInfo->ii_Expressions != NULL)
870         {
871                 TupleTableSlot *slot;
872                 ExprContext *econtext;
873
874                 /*
875                  * We will need to use FormIndexDatum to evaluate the index
876                  * expressions.  To do that, we need an EState, as well as a
877                  * TupleTableSlot to put the table tuples into.  The econtext's
878                  * scantuple has to point to that slot, too.
879                  */
880                 state->estate = CreateExecutorState();
881                 slot = MakeSingleTupleTableSlot(tupDesc);
882                 econtext = GetPerTupleExprContext(state->estate);
883                 econtext->ecxt_scantuple = slot;
884         }
885
886         /* Prepare SortSupport data for each column */
887         state->sortKeys = (SortSupport) palloc0(state->nKeys *
888                                                                                         sizeof(SortSupportData));
889
890         for (i = 0; i < state->nKeys; i++)
891         {
892                 SortSupport sortKey = state->sortKeys + i;
893                 ScanKey         scanKey = indexScanKey + i;
894                 int16           strategy;
895
896                 sortKey->ssup_cxt = CurrentMemoryContext;
897                 sortKey->ssup_collation = scanKey->sk_collation;
898                 sortKey->ssup_nulls_first =
899                         (scanKey->sk_flags & SK_BT_NULLS_FIRST) != 0;
900                 sortKey->ssup_attno = scanKey->sk_attno;
901                 /* Convey if abbreviation optimization is applicable in principle */
902                 sortKey->abbreviate = (i == 0);
903
904                 AssertState(sortKey->ssup_attno != 0);
905
906                 strategy = (scanKey->sk_flags & SK_BT_DESC) != 0 ?
907                         BTGreaterStrategyNumber : BTLessStrategyNumber;
908
909                 PrepareSortSupportFromIndexRel(indexRel, strategy, sortKey);
910         }
911
912         _bt_freeskey(indexScanKey);
913
914         MemoryContextSwitchTo(oldcontext);
915
916         return state;
917 }
918
919 Tuplesortstate *
920 tuplesort_begin_index_btree(Relation heapRel,
921                                                         Relation indexRel,
922                                                         bool enforceUnique,
923                                                         int workMem, bool randomAccess)
924 {
925         Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
926         ScanKey         indexScanKey;
927         MemoryContext oldcontext;
928         int                     i;
929
930         oldcontext = MemoryContextSwitchTo(state->sortcontext);
931
932 #ifdef TRACE_SORT
933         if (trace_sort)
934                 elog(LOG,
935                          "begin index sort: unique = %c, workMem = %d, randomAccess = %c",
936                          enforceUnique ? 't' : 'f',
937                          workMem, randomAccess ? 't' : 'f');
938 #endif
939
940         state->nKeys = RelationGetNumberOfAttributes(indexRel);
941
942         TRACE_POSTGRESQL_SORT_START(INDEX_SORT,
943                                                                 enforceUnique,
944                                                                 state->nKeys,
945                                                                 workMem,
946                                                                 randomAccess);
947
948         state->comparetup = comparetup_index_btree;
949         state->copytup = copytup_index;
950         state->writetup = writetup_index;
951         state->readtup = readtup_index;
952         state->abbrevNext = 10;
953
954         state->heapRel = heapRel;
955         state->indexRel = indexRel;
956         state->enforceUnique = enforceUnique;
957
958         indexScanKey = _bt_mkscankey_nodata(indexRel);
959         state->nKeys = RelationGetNumberOfAttributes(indexRel);
960
961         /* Prepare SortSupport data for each column */
962         state->sortKeys = (SortSupport) palloc0(state->nKeys *
963                                                                                         sizeof(SortSupportData));
964
965         for (i = 0; i < state->nKeys; i++)
966         {
967                 SortSupport sortKey = state->sortKeys + i;
968                 ScanKey         scanKey = indexScanKey + i;
969                 int16           strategy;
970
971                 sortKey->ssup_cxt = CurrentMemoryContext;
972                 sortKey->ssup_collation = scanKey->sk_collation;
973                 sortKey->ssup_nulls_first =
974                         (scanKey->sk_flags & SK_BT_NULLS_FIRST) != 0;
975                 sortKey->ssup_attno = scanKey->sk_attno;
976                 /* Convey if abbreviation optimization is applicable in principle */
977                 sortKey->abbreviate = (i == 0);
978
979                 AssertState(sortKey->ssup_attno != 0);
980
981                 strategy = (scanKey->sk_flags & SK_BT_DESC) != 0 ?
982                         BTGreaterStrategyNumber : BTLessStrategyNumber;
983
984                 PrepareSortSupportFromIndexRel(indexRel, strategy, sortKey);
985         }
986
987         _bt_freeskey(indexScanKey);
988
989         MemoryContextSwitchTo(oldcontext);
990
991         return state;
992 }
993
994 Tuplesortstate *
995 tuplesort_begin_index_hash(Relation heapRel,
996                                                    Relation indexRel,
997                                                    uint32 high_mask,
998                                                    uint32 low_mask,
999                                                    uint32 max_buckets,
1000                                                    int workMem, bool randomAccess)
1001 {
1002         Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
1003         MemoryContext oldcontext;
1004
1005         oldcontext = MemoryContextSwitchTo(state->sortcontext);
1006
1007 #ifdef TRACE_SORT
1008         if (trace_sort)
1009                 elog(LOG,
1010                          "begin index sort: high_mask = 0x%x, low_mask = 0x%x, "
1011                          "max_buckets = 0x%x, workMem = %d, randomAccess = %c",
1012                          high_mask,
1013                          low_mask,
1014                          max_buckets,
1015                          workMem, randomAccess ? 't' : 'f');
1016 #endif
1017
1018         state->nKeys = 1;                       /* Only one sort column, the hash code */
1019
1020         state->comparetup = comparetup_index_hash;
1021         state->copytup = copytup_index;
1022         state->writetup = writetup_index;
1023         state->readtup = readtup_index;
1024
1025         state->heapRel = heapRel;
1026         state->indexRel = indexRel;
1027
1028         state->high_mask = high_mask;
1029         state->low_mask = low_mask;
1030         state->max_buckets = max_buckets;
1031
1032         MemoryContextSwitchTo(oldcontext);
1033
1034         return state;
1035 }
1036
1037 Tuplesortstate *
1038 tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
1039                                           bool nullsFirstFlag,
1040                                           int workMem, bool randomAccess)
1041 {
1042         Tuplesortstate *state = tuplesort_begin_common(workMem, randomAccess);
1043         MemoryContext oldcontext;
1044         int16           typlen;
1045         bool            typbyval;
1046
1047         oldcontext = MemoryContextSwitchTo(state->sortcontext);
1048
1049 #ifdef TRACE_SORT
1050         if (trace_sort)
1051                 elog(LOG,
1052                          "begin datum sort: workMem = %d, randomAccess = %c",
1053                          workMem, randomAccess ? 't' : 'f');
1054 #endif
1055
1056         state->nKeys = 1;                       /* always a one-column sort */
1057
1058         TRACE_POSTGRESQL_SORT_START(DATUM_SORT,
1059                                                                 false,  /* no unique check */
1060                                                                 1,
1061                                                                 workMem,
1062                                                                 randomAccess);
1063
1064         state->comparetup = comparetup_datum;
1065         state->copytup = copytup_datum;
1066         state->writetup = writetup_datum;
1067         state->readtup = readtup_datum;
1068         state->abbrevNext = 10;
1069
1070         state->datumType = datumType;
1071
1072         /* lookup necessary attributes of the datum type */
1073         get_typlenbyval(datumType, &typlen, &typbyval);
1074         state->datumTypeLen = typlen;
1075         state->tuples = !typbyval;
1076
1077         /* Prepare SortSupport data */
1078         state->sortKeys = (SortSupport) palloc0(sizeof(SortSupportData));
1079
1080         state->sortKeys->ssup_cxt = CurrentMemoryContext;
1081         state->sortKeys->ssup_collation = sortCollation;
1082         state->sortKeys->ssup_nulls_first = nullsFirstFlag;
1083
1084         /*
1085          * Abbreviation is possible here only for by-reference types.  In theory,
1086          * a pass-by-value datatype could have an abbreviated form that is cheaper
1087          * to compare.  In a tuple sort, we could support that, because we can
1088          * always extract the original datum from the tuple is needed.  Here, we
1089          * can't, because a datum sort only stores a single copy of the datum; the
1090          * "tuple" field of each sortTuple is NULL.
1091          */
1092         state->sortKeys->abbreviate = !typbyval;
1093
1094         PrepareSortSupportFromOrderingOp(sortOperator, state->sortKeys);
1095
1096         /*
1097          * The "onlyKey" optimization cannot be used with abbreviated keys, since
1098          * tie-breaker comparisons may be required.  Typically, the optimization
1099          * is only of value to pass-by-value types anyway, whereas abbreviated
1100          * keys are typically only of value to pass-by-reference types.
1101          */
1102         if (!state->sortKeys->abbrev_converter)
1103                 state->onlyKey = state->sortKeys;
1104
1105         MemoryContextSwitchTo(oldcontext);
1106
1107         return state;
1108 }
1109
1110 /*
1111  * tuplesort_set_bound
1112  *
1113  *      Advise tuplesort that at most the first N result tuples are required.
1114  *
1115  * Must be called before inserting any tuples.  (Actually, we could allow it
1116  * as long as the sort hasn't spilled to disk, but there seems no need for
1117  * delayed calls at the moment.)
1118  *
1119  * This is a hint only. The tuplesort may still return more tuples than
1120  * requested.
1121  */
1122 void
1123 tuplesort_set_bound(Tuplesortstate *state, int64 bound)
1124 {
1125         /* Assert we're called before loading any tuples */
1126         Assert(state->status == TSS_INITIAL);
1127         Assert(state->memtupcount == 0);
1128         Assert(!state->bounded);
1129
1130 #ifdef DEBUG_BOUNDED_SORT
1131         /* Honor GUC setting that disables the feature (for easy testing) */
1132         if (!optimize_bounded_sort)
1133                 return;
1134 #endif
1135
1136         /* We want to be able to compute bound * 2, so limit the setting */
1137         if (bound > (int64) (INT_MAX / 2))
1138                 return;
1139
1140         state->bounded = true;
1141         state->bound = (int) bound;
1142
1143         /*
1144          * Bounded sorts are not an effective target for abbreviated key
1145          * optimization.  Disable by setting state to be consistent with no
1146          * abbreviation support.
1147          */
1148         state->sortKeys->abbrev_converter = NULL;
1149         if (state->sortKeys->abbrev_full_comparator)
1150                 state->sortKeys->comparator = state->sortKeys->abbrev_full_comparator;
1151
1152         /* Not strictly necessary, but be tidy */
1153         state->sortKeys->abbrev_abort = NULL;
1154         state->sortKeys->abbrev_full_comparator = NULL;
1155 }
1156
1157 /*
1158  * tuplesort_end
1159  *
1160  *      Release resources and clean up.
1161  *
1162  * NOTE: after calling this, any pointers returned by tuplesort_getXXX are
1163  * pointing to garbage.  Be careful not to attempt to use or free such
1164  * pointers afterwards!
1165  */
1166 void
1167 tuplesort_end(Tuplesortstate *state)
1168 {
1169         /* context swap probably not needed, but let's be safe */
1170         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
1171
1172 #ifdef TRACE_SORT
1173         long            spaceUsed;
1174
1175         if (state->tapeset)
1176                 spaceUsed = LogicalTapeSetBlocks(state->tapeset);
1177         else
1178                 spaceUsed = (state->allowedMem - state->availMem + 1023) / 1024;
1179 #endif
1180
1181         /*
1182          * Delete temporary "tape" files, if any.
1183          *
1184          * Note: want to include this in reported total cost of sort, hence need
1185          * for two #ifdef TRACE_SORT sections.
1186          */
1187         if (state->tapeset)
1188                 LogicalTapeSetClose(state->tapeset);
1189
1190 #ifdef TRACE_SORT
1191         if (trace_sort)
1192         {
1193                 if (state->tapeset)
1194                         elog(LOG, "external sort ended, %ld disk blocks used: %s",
1195                                  spaceUsed, pg_rusage_show(&state->ru_start));
1196                 else
1197                         elog(LOG, "internal sort ended, %ld KB used: %s",
1198                                  spaceUsed, pg_rusage_show(&state->ru_start));
1199         }
1200
1201         TRACE_POSTGRESQL_SORT_DONE(state->tapeset != NULL, spaceUsed);
1202 #else
1203
1204         /*
1205          * If you disabled TRACE_SORT, you can still probe sort__done, but you
1206          * ain't getting space-used stats.
1207          */
1208         TRACE_POSTGRESQL_SORT_DONE(state->tapeset != NULL, 0L);
1209 #endif
1210
1211         /* Free any execution state created for CLUSTER case */
1212         if (state->estate != NULL)
1213         {
1214                 ExprContext *econtext = GetPerTupleExprContext(state->estate);
1215
1216                 ExecDropSingleTupleTableSlot(econtext->ecxt_scantuple);
1217                 FreeExecutorState(state->estate);
1218         }
1219
1220         MemoryContextSwitchTo(oldcontext);
1221
1222         /*
1223          * Free the per-sort memory context, thereby releasing all working memory,
1224          * including the Tuplesortstate struct itself.
1225          */
1226         MemoryContextDelete(state->sortcontext);
1227 }
1228
1229 /*
1230  * Grow the memtuples[] array, if possible within our memory constraint.  We
1231  * must not exceed INT_MAX tuples in memory or the caller-provided memory
1232  * limit.  Return TRUE if we were able to enlarge the array, FALSE if not.
1233  *
1234  * Normally, at each increment we double the size of the array.  When doing
1235  * that would exceed a limit, we attempt one last, smaller increase (and then
1236  * clear the growmemtuples flag so we don't try any more).  That allows us to
1237  * use memory as fully as permitted; sticking to the pure doubling rule could
1238  * result in almost half going unused.  Because availMem moves around with
1239  * tuple addition/removal, we need some rule to prevent making repeated small
1240  * increases in memtupsize, which would just be useless thrashing.  The
1241  * growmemtuples flag accomplishes that and also prevents useless
1242  * recalculations in this function.
1243  */
1244 static bool
1245 grow_memtuples(Tuplesortstate *state)
1246 {
1247         int                     newmemtupsize;
1248         int                     memtupsize = state->memtupsize;
1249         int64           memNowUsed = state->allowedMem - state->availMem;
1250
1251         /* Forget it if we've already maxed out memtuples, per comment above */
1252         if (!state->growmemtuples)
1253                 return false;
1254
1255         /* Select new value of memtupsize */
1256         if (memNowUsed <= state->availMem)
1257         {
1258                 /*
1259                  * We've used no more than half of allowedMem; double our usage,
1260                  * clamping at INT_MAX tuples.
1261                  */
1262                 if (memtupsize < INT_MAX / 2)
1263                         newmemtupsize = memtupsize * 2;
1264                 else
1265                 {
1266                         newmemtupsize = INT_MAX;
1267                         state->growmemtuples = false;
1268                 }
1269         }
1270         else
1271         {
1272                 /*
1273                  * This will be the last increment of memtupsize.  Abandon doubling
1274                  * strategy and instead increase as much as we safely can.
1275                  *
1276                  * To stay within allowedMem, we can't increase memtupsize by more
1277                  * than availMem / sizeof(SortTuple) elements.  In practice, we want
1278                  * to increase it by considerably less, because we need to leave some
1279                  * space for the tuples to which the new array slots will refer.  We
1280                  * assume the new tuples will be about the same size as the tuples
1281                  * we've already seen, and thus we can extrapolate from the space
1282                  * consumption so far to estimate an appropriate new size for the
1283                  * memtuples array.  The optimal value might be higher or lower than
1284                  * this estimate, but it's hard to know that in advance.  We again
1285                  * clamp at INT_MAX tuples.
1286                  *
1287                  * This calculation is safe against enlarging the array so much that
1288                  * LACKMEM becomes true, because the memory currently used includes
1289                  * the present array; thus, there would be enough allowedMem for the
1290                  * new array elements even if no other memory were currently used.
1291                  *
1292                  * We do the arithmetic in float8, because otherwise the product of
1293                  * memtupsize and allowedMem could overflow.  Any inaccuracy in the
1294                  * result should be insignificant; but even if we computed a
1295                  * completely insane result, the checks below will prevent anything
1296                  * really bad from happening.
1297                  */
1298                 double          grow_ratio;
1299
1300                 grow_ratio = (double) state->allowedMem / (double) memNowUsed;
1301                 if (memtupsize * grow_ratio < INT_MAX)
1302                         newmemtupsize = (int) (memtupsize * grow_ratio);
1303                 else
1304                         newmemtupsize = INT_MAX;
1305
1306                 /* We won't make any further enlargement attempts */
1307                 state->growmemtuples = false;
1308         }
1309
1310         /* Must enlarge array by at least one element, else report failure */
1311         if (newmemtupsize <= memtupsize)
1312                 goto noalloc;
1313
1314         /*
1315          * On a 32-bit machine, allowedMem could exceed MaxAllocHugeSize.  Clamp
1316          * to ensure our request won't be rejected.  Note that we can easily
1317          * exhaust address space before facing this outcome.  (This is presently
1318          * impossible due to guc.c's MAX_KILOBYTES limitation on work_mem, but
1319          * don't rely on that at this distance.)
1320          */
1321         if ((Size) newmemtupsize >= MaxAllocHugeSize / sizeof(SortTuple))
1322         {
1323                 newmemtupsize = (int) (MaxAllocHugeSize / sizeof(SortTuple));
1324                 state->growmemtuples = false;   /* can't grow any more */
1325         }
1326
1327         /*
1328          * We need to be sure that we do not cause LACKMEM to become true, else
1329          * the space management algorithm will go nuts.  The code above should
1330          * never generate a dangerous request, but to be safe, check explicitly
1331          * that the array growth fits within availMem.  (We could still cause
1332          * LACKMEM if the memory chunk overhead associated with the memtuples
1333          * array were to increase.  That shouldn't happen because we chose the
1334          * initial array size large enough to ensure that palloc will be treating
1335          * both old and new arrays as separate chunks.  But we'll check LACKMEM
1336          * explicitly below just in case.)
1337          */
1338         if (state->availMem < (int64) ((newmemtupsize - memtupsize) * sizeof(SortTuple)))
1339                 goto noalloc;
1340
1341         /* OK, do it */
1342         FREEMEM(state, GetMemoryChunkSpace(state->memtuples));
1343         state->memtupsize = newmemtupsize;
1344         state->memtuples = (SortTuple *)
1345                 repalloc_huge(state->memtuples,
1346                                           state->memtupsize * sizeof(SortTuple));
1347         USEMEM(state, GetMemoryChunkSpace(state->memtuples));
1348         if (LACKMEM(state))
1349                 elog(ERROR, "unexpected out-of-memory situation in tuplesort");
1350         return true;
1351
1352 noalloc:
1353         /* If for any reason we didn't realloc, shut off future attempts */
1354         state->growmemtuples = false;
1355         return false;
1356 }
1357
1358 /*
1359  * Accept one tuple while collecting input data for sort.
1360  *
1361  * Note that the input data is always copied; the caller need not save it.
1362  */
1363 void
1364 tuplesort_puttupleslot(Tuplesortstate *state, TupleTableSlot *slot)
1365 {
1366         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
1367         SortTuple       stup;
1368
1369         /*
1370          * Copy the given tuple into memory we control, and decrease availMem.
1371          * Then call the common code.
1372          */
1373         COPYTUP(state, &stup, (void *) slot);
1374
1375         puttuple_common(state, &stup);
1376
1377         MemoryContextSwitchTo(oldcontext);
1378 }
1379
1380 /*
1381  * Accept one tuple while collecting input data for sort.
1382  *
1383  * Note that the input data is always copied; the caller need not save it.
1384  */
1385 void
1386 tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup)
1387 {
1388         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
1389         SortTuple       stup;
1390
1391         /*
1392          * Copy the given tuple into memory we control, and decrease availMem.
1393          * Then call the common code.
1394          */
1395         COPYTUP(state, &stup, (void *) tup);
1396
1397         puttuple_common(state, &stup);
1398
1399         MemoryContextSwitchTo(oldcontext);
1400 }
1401
1402 /*
1403  * Collect one index tuple while collecting input data for sort, building
1404  * it from caller-supplied values.
1405  */
1406 void
1407 tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
1408                                                           ItemPointer self, Datum *values,
1409                                                           bool *isnull)
1410 {
1411         MemoryContext oldcontext = MemoryContextSwitchTo(state->tuplecontext);
1412         SortTuple       stup;
1413         Datum           original;
1414         IndexTuple      tuple;
1415
1416         stup.tuple = index_form_tuple(RelationGetDescr(rel), values, isnull);
1417         tuple = ((IndexTuple) stup.tuple);
1418         tuple->t_tid = *self;
1419         USEMEM(state, GetMemoryChunkSpace(stup.tuple));
1420         /* set up first-column key value */
1421         original = index_getattr(tuple,
1422                                                          1,
1423                                                          RelationGetDescr(state->indexRel),
1424                                                          &stup.isnull1);
1425
1426         MemoryContextSwitchTo(state->sortcontext);
1427
1428         if (!state->sortKeys || !state->sortKeys->abbrev_converter || stup.isnull1)
1429         {
1430                 /*
1431                  * Store ordinary Datum representation, or NULL value.  If there is a
1432                  * converter it won't expect NULL values, and cost model is not
1433                  * required to account for NULL, so in that case we avoid calling
1434                  * converter and just set datum1 to zeroed representation (to be
1435                  * consistent, and to support cheap inequality tests for NULL
1436                  * abbreviated keys).
1437                  */
1438                 stup.datum1 = original;
1439         }
1440         else if (!consider_abort_common(state))
1441         {
1442                 /* Store abbreviated key representation */
1443                 stup.datum1 = state->sortKeys->abbrev_converter(original,
1444                                                                                                                 state->sortKeys);
1445         }
1446         else
1447         {
1448                 /* Abort abbreviation */
1449                 int                     i;
1450
1451                 stup.datum1 = original;
1452
1453                 /*
1454                  * Set state to be consistent with never trying abbreviation.
1455                  *
1456                  * Alter datum1 representation in already-copied tuples, so as to
1457                  * ensure a consistent representation (current tuple was just
1458                  * handled).  It does not matter if some dumped tuples are already
1459                  * sorted on tape, since serialized tuples lack abbreviated keys
1460                  * (TSS_BUILDRUNS state prevents control reaching here in any case).
1461                  */
1462                 for (i = 0; i < state->memtupcount; i++)
1463                 {
1464                         SortTuple  *mtup = &state->memtuples[i];
1465
1466                         tuple = mtup->tuple;
1467                         mtup->datum1 = index_getattr(tuple,
1468                                                                                  1,
1469                                                                                  RelationGetDescr(state->indexRel),
1470                                                                                  &mtup->isnull1);
1471                 }
1472         }
1473
1474         puttuple_common(state, &stup);
1475
1476         MemoryContextSwitchTo(oldcontext);
1477 }
1478
1479 /*
1480  * Accept one Datum while collecting input data for sort.
1481  *
1482  * If the Datum is pass-by-ref type, the value will be copied.
1483  */
1484 void
1485 tuplesort_putdatum(Tuplesortstate *state, Datum val, bool isNull)
1486 {
1487         MemoryContext oldcontext = MemoryContextSwitchTo(state->tuplecontext);
1488         SortTuple       stup;
1489
1490         /*
1491          * Pass-by-value types or null values are just stored directly in
1492          * stup.datum1 (and stup.tuple is not used and set to NULL).
1493          *
1494          * Non-null pass-by-reference values need to be copied into memory we
1495          * control, and possibly abbreviated. The copied value is pointed to by
1496          * stup.tuple and is treated as the canonical copy (e.g. to return via
1497          * tuplesort_getdatum or when writing to tape); stup.datum1 gets the
1498          * abbreviated value if abbreviation is happening, otherwise it's
1499          * identical to stup.tuple.
1500          */
1501
1502         if (isNull || !state->tuples)
1503         {
1504                 /*
1505                  * Set datum1 to zeroed representation for NULLs (to be consistent,
1506                  * and to support cheap inequality tests for NULL abbreviated keys).
1507                  */
1508                 stup.datum1 = !isNull ? val : (Datum) 0;
1509                 stup.isnull1 = isNull;
1510                 stup.tuple = NULL;              /* no separate storage */
1511                 MemoryContextSwitchTo(state->sortcontext);
1512         }
1513         else
1514         {
1515                 Datum           original = datumCopy(val, false, state->datumTypeLen);
1516
1517                 stup.isnull1 = false;
1518                 stup.tuple = DatumGetPointer(original);
1519                 USEMEM(state, GetMemoryChunkSpace(stup.tuple));
1520                 MemoryContextSwitchTo(state->sortcontext);
1521
1522                 if (!state->sortKeys->abbrev_converter)
1523                 {
1524                         stup.datum1 = original;
1525                 }
1526                 else if (!consider_abort_common(state))
1527                 {
1528                         /* Store abbreviated key representation */
1529                         stup.datum1 = state->sortKeys->abbrev_converter(original,
1530                                                                                                                         state->sortKeys);
1531                 }
1532                 else
1533                 {
1534                         /* Abort abbreviation */
1535                         int                     i;
1536
1537                         stup.datum1 = original;
1538
1539                         /*
1540                          * Set state to be consistent with never trying abbreviation.
1541                          *
1542                          * Alter datum1 representation in already-copied tuples, so as to
1543                          * ensure a consistent representation (current tuple was just
1544                          * handled).  It does not matter if some dumped tuples are already
1545                          * sorted on tape, since serialized tuples lack abbreviated keys
1546                          * (TSS_BUILDRUNS state prevents control reaching here in any
1547                          * case).
1548                          */
1549                         for (i = 0; i < state->memtupcount; i++)
1550                         {
1551                                 SortTuple  *mtup = &state->memtuples[i];
1552
1553                                 mtup->datum1 = PointerGetDatum(mtup->tuple);
1554                         }
1555                 }
1556         }
1557
1558         puttuple_common(state, &stup);
1559
1560         MemoryContextSwitchTo(oldcontext);
1561 }
1562
1563 /*
1564  * Shared code for tuple and datum cases.
1565  */
1566 static void
1567 puttuple_common(Tuplesortstate *state, SortTuple *tuple)
1568 {
1569         switch (state->status)
1570         {
1571                 case TSS_INITIAL:
1572
1573                         /*
1574                          * Save the tuple into the unsorted array.  First, grow the array
1575                          * as needed.  Note that we try to grow the array when there is
1576                          * still one free slot remaining --- if we fail, there'll still be
1577                          * room to store the incoming tuple, and then we'll switch to
1578                          * tape-based operation.
1579                          */
1580                         if (state->memtupcount >= state->memtupsize - 1)
1581                         {
1582                                 (void) grow_memtuples(state);
1583                                 Assert(state->memtupcount < state->memtupsize);
1584                         }
1585                         state->memtuples[state->memtupcount++] = *tuple;
1586
1587                         /*
1588                          * Check if it's time to switch over to a bounded heapsort. We do
1589                          * so if the input tuple count exceeds twice the desired tuple
1590                          * count (this is a heuristic for where heapsort becomes cheaper
1591                          * than a quicksort), or if we've just filled workMem and have
1592                          * enough tuples to meet the bound.
1593                          *
1594                          * Note that once we enter TSS_BOUNDED state we will always try to
1595                          * complete the sort that way.  In the worst case, if later input
1596                          * tuples are larger than earlier ones, this might cause us to
1597                          * exceed workMem significantly.
1598                          */
1599                         if (state->bounded &&
1600                                 (state->memtupcount > state->bound * 2 ||
1601                                  (state->memtupcount > state->bound && LACKMEM(state))))
1602                         {
1603 #ifdef TRACE_SORT
1604                                 if (trace_sort)
1605                                         elog(LOG, "switching to bounded heapsort at %d tuples: %s",
1606                                                  state->memtupcount,
1607                                                  pg_rusage_show(&state->ru_start));
1608 #endif
1609                                 make_bounded_heap(state);
1610                                 return;
1611                         }
1612
1613                         /*
1614                          * Done if we still fit in available memory and have array slots.
1615                          */
1616                         if (state->memtupcount < state->memtupsize && !LACKMEM(state))
1617                                 return;
1618
1619                         /*
1620                          * Nope; time to switch to tape-based operation.
1621                          */
1622                         inittapes(state);
1623
1624                         /*
1625                          * Dump tuples until we are back under the limit.
1626                          */
1627                         dumptuples(state, false);
1628                         break;
1629
1630                 case TSS_BOUNDED:
1631
1632                         /*
1633                          * We don't want to grow the array here, so check whether the new
1634                          * tuple can be discarded before putting it in.  This should be a
1635                          * good speed optimization, too, since when there are many more
1636                          * input tuples than the bound, most input tuples can be discarded
1637                          * with just this one comparison.  Note that because we currently
1638                          * have the sort direction reversed, we must check for <= not >=.
1639                          */
1640                         if (COMPARETUP(state, tuple, &state->memtuples[0]) <= 0)
1641                         {
1642                                 /* new tuple <= top of the heap, so we can discard it */
1643                                 free_sort_tuple(state, tuple);
1644                                 CHECK_FOR_INTERRUPTS();
1645                         }
1646                         else
1647                         {
1648                                 /* discard top of heap, replacing it with the new tuple */
1649                                 free_sort_tuple(state, &state->memtuples[0]);
1650                                 tuple->tupindex = 0;    /* not used */
1651                                 tuplesort_heap_replace_top(state, tuple, false);
1652                         }
1653                         break;
1654
1655                 case TSS_BUILDRUNS:
1656
1657                         /*
1658                          * Insert the tuple into the heap, with run number currentRun if
1659                          * it can go into the current run, else HEAP_RUN_NEXT.  The tuple
1660                          * can go into the current run if it is >= the first
1661                          * not-yet-output tuple.  (Actually, it could go into the current
1662                          * run if it is >= the most recently output tuple ... but that
1663                          * would require keeping around the tuple we last output, and it's
1664                          * simplest to let writetup free each tuple as soon as it's
1665                          * written.)
1666                          *
1667                          * Note that this only applies when:
1668                          *
1669                          * - currentRun is RUN_FIRST
1670                          *
1671                          * - Replacement selection is in use (typically it is never used).
1672                          *
1673                          * When these two conditions are not both true, all tuples are
1674                          * appended indifferently, much like the TSS_INITIAL case.
1675                          *
1676                          * There should always be room to store the incoming tuple.
1677                          */
1678                         Assert(!state->replaceActive || state->memtupcount > 0);
1679                         if (state->replaceActive &&
1680                                 COMPARETUP(state, tuple, &state->memtuples[0]) >= 0)
1681                         {
1682                                 Assert(state->currentRun == RUN_FIRST);
1683
1684                                 /*
1685                                  * Insert tuple into first, fully heapified run.
1686                                  *
1687                                  * Unlike classic replacement selection, which this module was
1688                                  * previously based on, only RUN_FIRST tuples are fully
1689                                  * heapified.  Any second/next run tuples are appended
1690                                  * indifferently.  While HEAP_RUN_NEXT tuples may be sifted
1691                                  * out of the way of first run tuples, COMPARETUP() will never
1692                                  * be called for the run's tuples during sifting (only our
1693                                  * initial COMPARETUP() call is required for the tuple, to
1694                                  * determine that the tuple does not belong in RUN_FIRST).
1695                                  */
1696                                 tuple->tupindex = state->currentRun;
1697                                 tuplesort_heap_insert(state, tuple, true);
1698                         }
1699                         else
1700                         {
1701                                 /*
1702                                  * Tuple was determined to not belong to heapified RUN_FIRST,
1703                                  * or replacement selection not in play.  Append the tuple to
1704                                  * memtuples indifferently.
1705                                  *
1706                                  * dumptuples() does not trust that the next run's tuples are
1707                                  * heapified.  Anything past the first run will always be
1708                                  * quicksorted even when replacement selection is initially
1709                                  * used.  (When it's never used, every tuple still takes this
1710                                  * path.)
1711                                  */
1712                                 tuple->tupindex = HEAP_RUN_NEXT;
1713                                 state->memtuples[state->memtupcount++] = *tuple;
1714                         }
1715
1716                         /*
1717                          * If we are over the memory limit, dump tuples till we're under.
1718                          */
1719                         dumptuples(state, false);
1720                         break;
1721
1722                 default:
1723                         elog(ERROR, "invalid tuplesort state");
1724                         break;
1725         }
1726 }
1727
1728 static bool
1729 consider_abort_common(Tuplesortstate *state)
1730 {
1731         Assert(state->sortKeys[0].abbrev_converter != NULL);
1732         Assert(state->sortKeys[0].abbrev_abort != NULL);
1733         Assert(state->sortKeys[0].abbrev_full_comparator != NULL);
1734
1735         /*
1736          * Check effectiveness of abbreviation optimization.  Consider aborting
1737          * when still within memory limit.
1738          */
1739         if (state->status == TSS_INITIAL &&
1740                 state->memtupcount >= state->abbrevNext)
1741         {
1742                 state->abbrevNext *= 2;
1743
1744                 /*
1745                  * Check opclass-supplied abbreviation abort routine.  It may indicate
1746                  * that abbreviation should not proceed.
1747                  */
1748                 if (!state->sortKeys->abbrev_abort(state->memtupcount,
1749                                                                                    state->sortKeys))
1750                         return false;
1751
1752                 /*
1753                  * Finally, restore authoritative comparator, and indicate that
1754                  * abbreviation is not in play by setting abbrev_converter to NULL
1755                  */
1756                 state->sortKeys[0].comparator = state->sortKeys[0].abbrev_full_comparator;
1757                 state->sortKeys[0].abbrev_converter = NULL;
1758                 /* Not strictly necessary, but be tidy */
1759                 state->sortKeys[0].abbrev_abort = NULL;
1760                 state->sortKeys[0].abbrev_full_comparator = NULL;
1761
1762                 /* Give up - expect original pass-by-value representation */
1763                 return true;
1764         }
1765
1766         return false;
1767 }
1768
1769 /*
1770  * All tuples have been provided; finish the sort.
1771  */
1772 void
1773 tuplesort_performsort(Tuplesortstate *state)
1774 {
1775         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
1776
1777 #ifdef TRACE_SORT
1778         if (trace_sort)
1779                 elog(LOG, "performsort starting: %s",
1780                          pg_rusage_show(&state->ru_start));
1781 #endif
1782
1783         switch (state->status)
1784         {
1785                 case TSS_INITIAL:
1786
1787                         /*
1788                          * We were able to accumulate all the tuples within the allowed
1789                          * amount of memory.  Just qsort 'em and we're done.
1790                          */
1791                         tuplesort_sort_memtuples(state);
1792                         state->current = 0;
1793                         state->eof_reached = false;
1794                         state->markpos_offset = 0;
1795                         state->markpos_eof = false;
1796                         state->status = TSS_SORTEDINMEM;
1797                         break;
1798
1799                 case TSS_BOUNDED:
1800
1801                         /*
1802                          * We were able to accumulate all the tuples required for output
1803                          * in memory, using a heap to eliminate excess tuples.  Now we
1804                          * have to transform the heap to a properly-sorted array.
1805                          */
1806                         sort_bounded_heap(state);
1807                         state->current = 0;
1808                         state->eof_reached = false;
1809                         state->markpos_offset = 0;
1810                         state->markpos_eof = false;
1811                         state->status = TSS_SORTEDINMEM;
1812                         break;
1813
1814                 case TSS_BUILDRUNS:
1815
1816                         /*
1817                          * Finish tape-based sort.  First, flush all tuples remaining in
1818                          * memory out to tape; then merge until we have a single remaining
1819                          * run (or, if !randomAccess, one run per tape). Note that
1820                          * mergeruns sets the correct state->status.
1821                          */
1822                         dumptuples(state, true);
1823                         mergeruns(state);
1824                         state->eof_reached = false;
1825                         state->markpos_block = 0L;
1826                         state->markpos_offset = 0;
1827                         state->markpos_eof = false;
1828                         break;
1829
1830                 default:
1831                         elog(ERROR, "invalid tuplesort state");
1832                         break;
1833         }
1834
1835 #ifdef TRACE_SORT
1836         if (trace_sort)
1837         {
1838                 if (state->status == TSS_FINALMERGE)
1839                         elog(LOG, "performsort done (except %d-way final merge): %s",
1840                                  state->activeTapes,
1841                                  pg_rusage_show(&state->ru_start));
1842                 else
1843                         elog(LOG, "performsort done: %s",
1844                                  pg_rusage_show(&state->ru_start));
1845         }
1846 #endif
1847
1848         MemoryContextSwitchTo(oldcontext);
1849 }
1850
1851 /*
1852  * Internal routine to fetch the next tuple in either forward or back
1853  * direction into *stup.  Returns FALSE if no more tuples.
1854  * Returned tuple belongs to tuplesort memory context, and must not be freed
1855  * by caller.  Note that fetched tuple is stored in memory that may be
1856  * recycled by any future fetch.
1857  */
1858 static bool
1859 tuplesort_gettuple_common(Tuplesortstate *state, bool forward,
1860                                                   SortTuple *stup)
1861 {
1862         unsigned int tuplen;
1863         size_t          nmoved;
1864
1865         switch (state->status)
1866         {
1867                 case TSS_SORTEDINMEM:
1868                         Assert(forward || state->randomAccess);
1869                         Assert(!state->slabAllocatorUsed);
1870                         if (forward)
1871                         {
1872                                 if (state->current < state->memtupcount)
1873                                 {
1874                                         *stup = state->memtuples[state->current++];
1875                                         return true;
1876                                 }
1877                                 state->eof_reached = true;
1878
1879                                 /*
1880                                  * Complain if caller tries to retrieve more tuples than
1881                                  * originally asked for in a bounded sort.  This is because
1882                                  * returning EOF here might be the wrong thing.
1883                                  */
1884                                 if (state->bounded && state->current >= state->bound)
1885                                         elog(ERROR, "retrieved too many tuples in a bounded sort");
1886
1887                                 return false;
1888                         }
1889                         else
1890                         {
1891                                 if (state->current <= 0)
1892                                         return false;
1893
1894                                 /*
1895                                  * if all tuples are fetched already then we return last
1896                                  * tuple, else - tuple before last returned.
1897                                  */
1898                                 if (state->eof_reached)
1899                                         state->eof_reached = false;
1900                                 else
1901                                 {
1902                                         state->current--;       /* last returned tuple */
1903                                         if (state->current <= 0)
1904                                                 return false;
1905                                 }
1906                                 *stup = state->memtuples[state->current - 1];
1907                                 return true;
1908                         }
1909                         break;
1910
1911                 case TSS_SORTEDONTAPE:
1912                         Assert(forward || state->randomAccess);
1913                         Assert(state->slabAllocatorUsed);
1914
1915                         /*
1916                          * The slot that held the tuple that we returned in previous
1917                          * gettuple call can now be reused.
1918                          */
1919                         if (state->lastReturnedTuple)
1920                         {
1921                                 RELEASE_SLAB_SLOT(state, state->lastReturnedTuple);
1922                                 state->lastReturnedTuple = NULL;
1923                         }
1924
1925                         if (forward)
1926                         {
1927                                 if (state->eof_reached)
1928                                         return false;
1929
1930                                 if ((tuplen = getlen(state, state->result_tape, true)) != 0)
1931                                 {
1932                                         READTUP(state, stup, state->result_tape, tuplen);
1933
1934                                         /*
1935                                          * Remember the tuple we return, so that we can recycle
1936                                          * its memory on next call.  (This can be NULL, in the
1937                                          * !state->tuples case).
1938                                          */
1939                                         state->lastReturnedTuple = stup->tuple;
1940
1941                                         return true;
1942                                 }
1943                                 else
1944                                 {
1945                                         state->eof_reached = true;
1946                                         return false;
1947                                 }
1948                         }
1949
1950                         /*
1951                          * Backward.
1952                          *
1953                          * if all tuples are fetched already then we return last tuple,
1954                          * else - tuple before last returned.
1955                          */
1956                         if (state->eof_reached)
1957                         {
1958                                 /*
1959                                  * Seek position is pointing just past the zero tuplen at the
1960                                  * end of file; back up to fetch last tuple's ending length
1961                                  * word.  If seek fails we must have a completely empty file.
1962                                  */
1963                                 nmoved = LogicalTapeBackspace(state->tapeset,
1964                                                                                           state->result_tape,
1965                                                                                           2 * sizeof(unsigned int));
1966                                 if (nmoved == 0)
1967                                         return false;
1968                                 else if (nmoved != 2 * sizeof(unsigned int))
1969                                         elog(ERROR, "unexpected tape position");
1970                                 state->eof_reached = false;
1971                         }
1972                         else
1973                         {
1974                                 /*
1975                                  * Back up and fetch previously-returned tuple's ending length
1976                                  * word.  If seek fails, assume we are at start of file.
1977                                  */
1978                                 nmoved = LogicalTapeBackspace(state->tapeset,
1979                                                                                           state->result_tape,
1980                                                                                           sizeof(unsigned int));
1981                                 if (nmoved == 0)
1982                                         return false;
1983                                 else if (nmoved != sizeof(unsigned int))
1984                                         elog(ERROR, "unexpected tape position");
1985                                 tuplen = getlen(state, state->result_tape, false);
1986
1987                                 /*
1988                                  * Back up to get ending length word of tuple before it.
1989                                  */
1990                                 nmoved = LogicalTapeBackspace(state->tapeset,
1991                                                                                           state->result_tape,
1992                                                                                           tuplen + 2 * sizeof(unsigned int));
1993                                 if (nmoved == tuplen + sizeof(unsigned int))
1994                                 {
1995                                         /*
1996                                          * We backed up over the previous tuple, but there was no
1997                                          * ending length word before it.  That means that the prev
1998                                          * tuple is the first tuple in the file.  It is now the
1999                                          * next to read in forward direction (not obviously right,
2000                                          * but that is what in-memory case does).
2001                                          */
2002                                         return false;
2003                                 }
2004                                 else if (nmoved != tuplen + 2 * sizeof(unsigned int))
2005                                         elog(ERROR, "bogus tuple length in backward scan");
2006                         }
2007
2008                         tuplen = getlen(state, state->result_tape, false);
2009
2010                         /*
2011                          * Now we have the length of the prior tuple, back up and read it.
2012                          * Note: READTUP expects we are positioned after the initial
2013                          * length word of the tuple, so back up to that point.
2014                          */
2015                         nmoved = LogicalTapeBackspace(state->tapeset,
2016                                                                                   state->result_tape,
2017                                                                                   tuplen);
2018                         if (nmoved != tuplen)
2019                                 elog(ERROR, "bogus tuple length in backward scan");
2020                         READTUP(state, stup, state->result_tape, tuplen);
2021
2022                         /*
2023                          * Remember the tuple we return, so that we can recycle its memory
2024                          * on next call. (This can be NULL, in the Datum case).
2025                          */
2026                         state->lastReturnedTuple = stup->tuple;
2027
2028                         return true;
2029
2030                 case TSS_FINALMERGE:
2031                         Assert(forward);
2032                         /* We are managing memory ourselves, with the slab allocator. */
2033                         Assert(state->slabAllocatorUsed);
2034
2035                         /*
2036                          * The slab slot holding the tuple that we returned in previous
2037                          * gettuple call can now be reused.
2038                          */
2039                         if (state->lastReturnedTuple)
2040                         {
2041                                 RELEASE_SLAB_SLOT(state, state->lastReturnedTuple);
2042                                 state->lastReturnedTuple = NULL;
2043                         }
2044
2045                         /*
2046                          * This code should match the inner loop of mergeonerun().
2047                          */
2048                         if (state->memtupcount > 0)
2049                         {
2050                                 int                     srcTape = state->memtuples[0].tupindex;
2051                                 SortTuple       newtup;
2052
2053                                 *stup = state->memtuples[0];
2054
2055                                 /*
2056                                  * Remember the tuple we return, so that we can recycle its
2057                                  * memory on next call. (This can be NULL, in the Datum case).
2058                                  */
2059                                 state->lastReturnedTuple = stup->tuple;
2060
2061                                 /*
2062                                  * Pull next tuple from tape, and replace the returned tuple
2063                                  * at top of the heap with it.
2064                                  */
2065                                 if (!mergereadnext(state, srcTape, &newtup))
2066                                 {
2067                                         /*
2068                                          * If no more data, we've reached end of run on this tape.
2069                                          * Remove the top node from the heap.
2070                                          */
2071                                         tuplesort_heap_delete_top(state, false);
2072
2073                                         /*
2074                                          * Rewind to free the read buffer.  It'd go away at the
2075                                          * end of the sort anyway, but better to release the
2076                                          * memory early.
2077                                          */
2078                                         LogicalTapeRewindForWrite(state->tapeset, srcTape);
2079                                         return true;
2080                                 }
2081                                 newtup.tupindex = srcTape;
2082                                 tuplesort_heap_replace_top(state, &newtup, false);
2083                                 return true;
2084                         }
2085                         return false;
2086
2087                 default:
2088                         elog(ERROR, "invalid tuplesort state");
2089                         return false;           /* keep compiler quiet */
2090         }
2091 }
2092
2093 /*
2094  * Fetch the next tuple in either forward or back direction.
2095  * If successful, put tuple in slot and return TRUE; else, clear the slot
2096  * and return FALSE.
2097  *
2098  * Caller may optionally be passed back abbreviated value (on TRUE return
2099  * value) when abbreviation was used, which can be used to cheaply avoid
2100  * equality checks that might otherwise be required.  Caller can safely make a
2101  * determination of "non-equal tuple" based on simple binary inequality.  A
2102  * NULL value in leading attribute will set abbreviated value to zeroed
2103  * representation, which caller may rely on in abbreviated inequality check.
2104  *
2105  * If copy is true, the slot receives a copied tuple that will stay valid
2106  * regardless of future manipulations of the tuplesort's state.  Memory is
2107  * owned by the caller.  If copy is false, the slot will just receive a
2108  * pointer to a tuple held within the tuplesort, which is more efficient, but
2109  * only safe for callers that are prepared to have any subsequent manipulation
2110  * of the tuplesort's state invalidate slot contents.
2111  */
2112 bool
2113 tuplesort_gettupleslot(Tuplesortstate *state, bool forward, bool copy,
2114                                            TupleTableSlot *slot, Datum *abbrev)
2115 {
2116         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
2117         SortTuple       stup;
2118
2119         if (!tuplesort_gettuple_common(state, forward, &stup))
2120                 stup.tuple = NULL;
2121
2122         MemoryContextSwitchTo(oldcontext);
2123
2124         if (stup.tuple)
2125         {
2126                 /* Record abbreviated key for caller */
2127                 if (state->sortKeys->abbrev_converter && abbrev)
2128                         *abbrev = stup.datum1;
2129
2130                 if (copy)
2131                         stup.tuple = heap_copy_minimal_tuple((MinimalTuple) stup.tuple);
2132
2133                 ExecStoreMinimalTuple((MinimalTuple) stup.tuple, slot, copy);
2134                 return true;
2135         }
2136         else
2137         {
2138                 ExecClearTuple(slot);
2139                 return false;
2140         }
2141 }
2142
2143 /*
2144  * Fetch the next tuple in either forward or back direction.
2145  * Returns NULL if no more tuples.  Returned tuple belongs to tuplesort memory
2146  * context, and must not be freed by caller.  Caller may not rely on tuple
2147  * remaining valid after any further manipulation of tuplesort.
2148  */
2149 HeapTuple
2150 tuplesort_getheaptuple(Tuplesortstate *state, bool forward)
2151 {
2152         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
2153         SortTuple       stup;
2154
2155         if (!tuplesort_gettuple_common(state, forward, &stup))
2156                 stup.tuple = NULL;
2157
2158         MemoryContextSwitchTo(oldcontext);
2159
2160         return stup.tuple;
2161 }
2162
2163 /*
2164  * Fetch the next index tuple in either forward or back direction.
2165  * Returns NULL if no more tuples.  Returned tuple belongs to tuplesort memory
2166  * context, and must not be freed by caller.  Caller may not rely on tuple
2167  * remaining valid after any further manipulation of tuplesort.
2168  */
2169 IndexTuple
2170 tuplesort_getindextuple(Tuplesortstate *state, bool forward)
2171 {
2172         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
2173         SortTuple       stup;
2174
2175         if (!tuplesort_gettuple_common(state, forward, &stup))
2176                 stup.tuple = NULL;
2177
2178         MemoryContextSwitchTo(oldcontext);
2179
2180         return (IndexTuple) stup.tuple;
2181 }
2182
2183 /*
2184  * Fetch the next Datum in either forward or back direction.
2185  * Returns FALSE if no more datums.
2186  *
2187  * If the Datum is pass-by-ref type, the returned value is freshly palloc'd
2188  * and is now owned by the caller (this differs from similar routines for
2189  * other types of tuplesorts).
2190  *
2191  * Caller may optionally be passed back abbreviated value (on TRUE return
2192  * value) when abbreviation was used, which can be used to cheaply avoid
2193  * equality checks that might otherwise be required.  Caller can safely make a
2194  * determination of "non-equal tuple" based on simple binary inequality.  A
2195  * NULL value will have a zeroed abbreviated value representation, which caller
2196  * may rely on in abbreviated inequality check.
2197  */
2198 bool
2199 tuplesort_getdatum(Tuplesortstate *state, bool forward,
2200                                    Datum *val, bool *isNull, Datum *abbrev)
2201 {
2202         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
2203         SortTuple       stup;
2204
2205         if (!tuplesort_gettuple_common(state, forward, &stup))
2206         {
2207                 MemoryContextSwitchTo(oldcontext);
2208                 return false;
2209         }
2210
2211         /* Record abbreviated key for caller */
2212         if (state->sortKeys->abbrev_converter && abbrev)
2213                 *abbrev = stup.datum1;
2214
2215         if (stup.isnull1 || !state->tuples)
2216         {
2217                 *val = stup.datum1;
2218                 *isNull = stup.isnull1;
2219         }
2220         else
2221         {
2222                 /* use stup.tuple because stup.datum1 may be an abbreviation */
2223                 *val = datumCopy(PointerGetDatum(stup.tuple), false, state->datumTypeLen);
2224                 *isNull = false;
2225         }
2226
2227         MemoryContextSwitchTo(oldcontext);
2228
2229         return true;
2230 }
2231
2232 /*
2233  * Advance over N tuples in either forward or back direction,
2234  * without returning any data.  N==0 is a no-op.
2235  * Returns TRUE if successful, FALSE if ran out of tuples.
2236  */
2237 bool
2238 tuplesort_skiptuples(Tuplesortstate *state, int64 ntuples, bool forward)
2239 {
2240         MemoryContext oldcontext;
2241
2242         /*
2243          * We don't actually support backwards skip yet, because no callers need
2244          * it.  The API is designed to allow for that later, though.
2245          */
2246         Assert(forward);
2247         Assert(ntuples >= 0);
2248
2249         switch (state->status)
2250         {
2251                 case TSS_SORTEDINMEM:
2252                         if (state->memtupcount - state->current >= ntuples)
2253                         {
2254                                 state->current += ntuples;
2255                                 return true;
2256                         }
2257                         state->current = state->memtupcount;
2258                         state->eof_reached = true;
2259
2260                         /*
2261                          * Complain if caller tries to retrieve more tuples than
2262                          * originally asked for in a bounded sort.  This is because
2263                          * returning EOF here might be the wrong thing.
2264                          */
2265                         if (state->bounded && state->current >= state->bound)
2266                                 elog(ERROR, "retrieved too many tuples in a bounded sort");
2267
2268                         return false;
2269
2270                 case TSS_SORTEDONTAPE:
2271                 case TSS_FINALMERGE:
2272
2273                         /*
2274                          * We could probably optimize these cases better, but for now it's
2275                          * not worth the trouble.
2276                          */
2277                         oldcontext = MemoryContextSwitchTo(state->sortcontext);
2278                         while (ntuples-- > 0)
2279                         {
2280                                 SortTuple       stup;
2281
2282                                 if (!tuplesort_gettuple_common(state, forward, &stup))
2283                                 {
2284                                         MemoryContextSwitchTo(oldcontext);
2285                                         return false;
2286                                 }
2287                                 CHECK_FOR_INTERRUPTS();
2288                         }
2289                         MemoryContextSwitchTo(oldcontext);
2290                         return true;
2291
2292                 default:
2293                         elog(ERROR, "invalid tuplesort state");
2294                         return false;           /* keep compiler quiet */
2295         }
2296 }
2297
2298 /*
2299  * tuplesort_merge_order - report merge order we'll use for given memory
2300  * (note: "merge order" just means the number of input tapes in the merge).
2301  *
2302  * This is exported for use by the planner.  allowedMem is in bytes.
2303  */
2304 int
2305 tuplesort_merge_order(int64 allowedMem)
2306 {
2307         int                     mOrder;
2308
2309         /*
2310          * We need one tape for each merge input, plus another one for the output,
2311          * and each of these tapes needs buffer space.  In addition we want
2312          * MERGE_BUFFER_SIZE workspace per input tape (but the output tape doesn't
2313          * count).
2314          *
2315          * Note: you might be thinking we need to account for the memtuples[]
2316          * array in this calculation, but we effectively treat that as part of the
2317          * MERGE_BUFFER_SIZE workspace.
2318          */
2319         mOrder = (allowedMem - TAPE_BUFFER_OVERHEAD) /
2320                 (MERGE_BUFFER_SIZE + TAPE_BUFFER_OVERHEAD);
2321
2322         /*
2323          * Even in minimum memory, use at least a MINORDER merge.  On the other
2324          * hand, even when we have lots of memory, do not use more than a MAXORDER
2325          * merge.  Tapes are pretty cheap, but they're not entirely free.  Each
2326          * additional tape reduces the amount of memory available to build runs,
2327          * which in turn can cause the same sort to need more runs, which makes
2328          * merging slower even if it can still be done in a single pass.  Also,
2329          * high order merges are quite slow due to CPU cache effects; it can be
2330          * faster to pay the I/O cost of a polyphase merge than to perform a
2331          * single merge pass across many hundreds of tapes.
2332          */
2333         mOrder = Max(mOrder, MINORDER);
2334         mOrder = Min(mOrder, MAXORDER);
2335
2336         return mOrder;
2337 }
2338
2339 /*
2340  * useselection - determine algorithm to use to sort first run.
2341  *
2342  * It can sometimes be useful to use the replacement selection algorithm if it
2343  * results in one large run, and there is little available workMem.  See
2344  * remarks on RUN_SECOND optimization within dumptuples().
2345  */
2346 static bool
2347 useselection(Tuplesortstate *state)
2348 {
2349         /*
2350          * memtupsize might be noticeably higher than memtupcount here in atypical
2351          * cases.  It seems slightly preferable to not allow recent outliers to
2352          * impact this determination.  Note that caller's trace_sort output
2353          * reports memtupcount instead.
2354          */
2355         if (state->memtupsize <= replacement_sort_tuples)
2356                 return true;
2357
2358         return false;
2359 }
2360
2361 /*
2362  * inittapes - initialize for tape sorting.
2363  *
2364  * This is called only if we have found we don't have room to sort in memory.
2365  */
2366 static void
2367 inittapes(Tuplesortstate *state)
2368 {
2369         int                     maxTapes,
2370                                 j;
2371         int64           tapeSpace;
2372
2373         /* Compute number of tapes to use: merge order plus 1 */
2374         maxTapes = tuplesort_merge_order(state->allowedMem) + 1;
2375
2376         state->maxTapes = maxTapes;
2377         state->tapeRange = maxTapes - 1;
2378
2379 #ifdef TRACE_SORT
2380         if (trace_sort)
2381                 elog(LOG, "switching to external sort with %d tapes: %s",
2382                          maxTapes, pg_rusage_show(&state->ru_start));
2383 #endif
2384
2385         /*
2386          * Decrease availMem to reflect the space needed for tape buffers, when
2387          * writing the initial runs; but don't decrease it to the point that we
2388          * have no room for tuples.  (That case is only likely to occur if sorting
2389          * pass-by-value Datums; in all other scenarios the memtuples[] array is
2390          * unlikely to occupy more than half of allowedMem.  In the pass-by-value
2391          * case it's not important to account for tuple space, so we don't care if
2392          * LACKMEM becomes inaccurate.)
2393          */
2394         tapeSpace = (int64) maxTapes * TAPE_BUFFER_OVERHEAD;
2395
2396         if (tapeSpace + GetMemoryChunkSpace(state->memtuples) < state->allowedMem)
2397                 USEMEM(state, tapeSpace);
2398
2399         /*
2400          * Make sure that the temp file(s) underlying the tape set are created in
2401          * suitable temp tablespaces.
2402          */
2403         PrepareTempTablespaces();
2404
2405         /*
2406          * Create the tape set and allocate the per-tape data arrays.
2407          */
2408         state->tapeset = LogicalTapeSetCreate(maxTapes);
2409
2410         state->mergeactive = (bool *) palloc0(maxTapes * sizeof(bool));
2411         state->tp_fib = (int *) palloc0(maxTapes * sizeof(int));
2412         state->tp_runs = (int *) palloc0(maxTapes * sizeof(int));
2413         state->tp_dummy = (int *) palloc0(maxTapes * sizeof(int));
2414         state->tp_tapenum = (int *) palloc0(maxTapes * sizeof(int));
2415
2416         /*
2417          * Give replacement selection a try based on user setting.  There will be
2418          * a switch to a simple hybrid sort-merge strategy after the first run
2419          * (iff we could not output one long run).
2420          */
2421         state->replaceActive = useselection(state);
2422
2423         if (state->replaceActive)
2424         {
2425                 /*
2426                  * Convert the unsorted contents of memtuples[] into a heap. Each
2427                  * tuple is marked as belonging to run number zero.
2428                  *
2429                  * NOTE: we pass false for checkIndex since there's no point in
2430                  * comparing indexes in this step, even though we do intend the
2431                  * indexes to be part of the sort key...
2432                  */
2433                 int                     ntuples = state->memtupcount;
2434
2435 #ifdef TRACE_SORT
2436                 if (trace_sort)
2437                         elog(LOG, "replacement selection will sort %d first run tuples",
2438                                  state->memtupcount);
2439 #endif
2440                 state->memtupcount = 0; /* make the heap empty */
2441
2442                 for (j = 0; j < ntuples; j++)
2443                 {
2444                         /* Must copy source tuple to avoid possible overwrite */
2445                         SortTuple       stup = state->memtuples[j];
2446
2447                         stup.tupindex = RUN_FIRST;
2448                         tuplesort_heap_insert(state, &stup, false);
2449                 }
2450                 Assert(state->memtupcount == ntuples);
2451         }
2452
2453         state->currentRun = RUN_FIRST;
2454
2455         /*
2456          * Initialize variables of Algorithm D (step D1).
2457          */
2458         for (j = 0; j < maxTapes; j++)
2459         {
2460                 state->tp_fib[j] = 1;
2461                 state->tp_runs[j] = 0;
2462                 state->tp_dummy[j] = 1;
2463                 state->tp_tapenum[j] = j;
2464         }
2465         state->tp_fib[state->tapeRange] = 0;
2466         state->tp_dummy[state->tapeRange] = 0;
2467
2468         state->Level = 1;
2469         state->destTape = 0;
2470
2471         state->status = TSS_BUILDRUNS;
2472 }
2473
2474 /*
2475  * selectnewtape -- select new tape for new initial run.
2476  *
2477  * This is called after finishing a run when we know another run
2478  * must be started.  This implements steps D3, D4 of Algorithm D.
2479  */
2480 static void
2481 selectnewtape(Tuplesortstate *state)
2482 {
2483         int                     j;
2484         int                     a;
2485
2486         /* Step D3: advance j (destTape) */
2487         if (state->tp_dummy[state->destTape] < state->tp_dummy[state->destTape + 1])
2488         {
2489                 state->destTape++;
2490                 return;
2491         }
2492         if (state->tp_dummy[state->destTape] != 0)
2493         {
2494                 state->destTape = 0;
2495                 return;
2496         }
2497
2498         /* Step D4: increase level */
2499         state->Level++;
2500         a = state->tp_fib[0];
2501         for (j = 0; j < state->tapeRange; j++)
2502         {
2503                 state->tp_dummy[j] = a + state->tp_fib[j + 1] - state->tp_fib[j];
2504                 state->tp_fib[j] = a + state->tp_fib[j + 1];
2505         }
2506         state->destTape = 0;
2507 }
2508
2509 /*
2510  * Initialize the slab allocation arena, for the given number of slots.
2511  */
2512 static void
2513 init_slab_allocator(Tuplesortstate *state, int numSlots)
2514 {
2515         if (numSlots > 0)
2516         {
2517                 char       *p;
2518                 int                     i;
2519
2520                 state->slabMemoryBegin = palloc(numSlots * SLAB_SLOT_SIZE);
2521                 state->slabMemoryEnd = state->slabMemoryBegin +
2522                         numSlots * SLAB_SLOT_SIZE;
2523                 state->slabFreeHead = (SlabSlot *) state->slabMemoryBegin;
2524                 USEMEM(state, numSlots * SLAB_SLOT_SIZE);
2525
2526                 p = state->slabMemoryBegin;
2527                 for (i = 0; i < numSlots - 1; i++)
2528                 {
2529                         ((SlabSlot *) p)->nextfree = (SlabSlot *) (p + SLAB_SLOT_SIZE);
2530                         p += SLAB_SLOT_SIZE;
2531                 }
2532                 ((SlabSlot *) p)->nextfree = NULL;
2533         }
2534         else
2535         {
2536                 state->slabMemoryBegin = state->slabMemoryEnd = NULL;
2537                 state->slabFreeHead = NULL;
2538         }
2539         state->slabAllocatorUsed = true;
2540 }
2541
2542 /*
2543  * mergeruns -- merge all the completed initial runs.
2544  *
2545  * This implements steps D5, D6 of Algorithm D.  All input data has
2546  * already been written to initial runs on tape (see dumptuples).
2547  */
2548 static void
2549 mergeruns(Tuplesortstate *state)
2550 {
2551         int                     tapenum,
2552                                 svTape,
2553                                 svRuns,
2554                                 svDummy;
2555         int                     numTapes;
2556         int                     numInputTapes;
2557
2558         Assert(state->status == TSS_BUILDRUNS);
2559         Assert(state->memtupcount == 0);
2560
2561         if (state->sortKeys != NULL && state->sortKeys->abbrev_converter != NULL)
2562         {
2563                 /*
2564                  * If there are multiple runs to be merged, when we go to read back
2565                  * tuples from disk, abbreviated keys will not have been stored, and
2566                  * we don't care to regenerate them.  Disable abbreviation from this
2567                  * point on.
2568                  */
2569                 state->sortKeys->abbrev_converter = NULL;
2570                 state->sortKeys->comparator = state->sortKeys->abbrev_full_comparator;
2571
2572                 /* Not strictly necessary, but be tidy */
2573                 state->sortKeys->abbrev_abort = NULL;
2574                 state->sortKeys->abbrev_full_comparator = NULL;
2575         }
2576
2577         /*
2578          * Reset tuple memory.  We've freed all the tuples that we previously
2579          * allocated.  We will use the slab allocator from now on.
2580          */
2581         MemoryContextDelete(state->tuplecontext);
2582         state->tuplecontext = NULL;
2583
2584         /*
2585          * We no longer need a large memtuples array.  (We will allocate a smaller
2586          * one for the heap later.)
2587          */
2588         FREEMEM(state, GetMemoryChunkSpace(state->memtuples));
2589         pfree(state->memtuples);
2590         state->memtuples = NULL;
2591
2592         /*
2593          * If we had fewer runs than tapes, refund the memory that we imagined we
2594          * would need for the tape buffers of the unused tapes.
2595          *
2596          * numTapes and numInputTapes reflect the actual number of tapes we will
2597          * use.  Note that the output tape's tape number is maxTapes - 1, so the
2598          * tape numbers of the used tapes are not consecutive, and you cannot just
2599          * loop from 0 to numTapes to visit all used tapes!
2600          */
2601         if (state->Level == 1)
2602         {
2603                 numInputTapes = state->currentRun;
2604                 numTapes = numInputTapes + 1;
2605                 FREEMEM(state, (state->maxTapes - numTapes) * TAPE_BUFFER_OVERHEAD);
2606         }
2607         else
2608         {
2609                 numInputTapes = state->tapeRange;
2610                 numTapes = state->maxTapes;
2611         }
2612
2613         /*
2614          * Initialize the slab allocator.  We need one slab slot per input tape,
2615          * for the tuples in the heap, plus one to hold the tuple last returned
2616          * from tuplesort_gettuple.  (If we're sorting pass-by-val Datums,
2617          * however, we don't need to do allocate anything.)
2618          *
2619          * From this point on, we no longer use the USEMEM()/LACKMEM() mechanism
2620          * to track memory usage of individual tuples.
2621          */
2622         if (state->tuples)
2623                 init_slab_allocator(state, numInputTapes + 1);
2624         else
2625                 init_slab_allocator(state, 0);
2626
2627         /*
2628          * If we produced only one initial run (quite likely if the total data
2629          * volume is between 1X and 2X workMem when replacement selection is used,
2630          * but something we particularly count on when input is presorted), we can
2631          * just use that tape as the finished output, rather than doing a useless
2632          * merge.  (This obvious optimization is not in Knuth's algorithm.)
2633          */
2634         if (state->currentRun == RUN_SECOND)
2635         {
2636                 state->result_tape = state->tp_tapenum[state->destTape];
2637                 /* must freeze and rewind the finished output tape */
2638                 LogicalTapeFreeze(state->tapeset, state->result_tape);
2639                 state->status = TSS_SORTEDONTAPE;
2640                 return;
2641         }
2642
2643         /*
2644          * Allocate a new 'memtuples' array, for the heap.  It will hold one tuple
2645          * from each input tape.
2646          */
2647         state->memtupsize = numInputTapes;
2648         state->memtuples = (SortTuple *) palloc(numInputTapes * sizeof(SortTuple));
2649         USEMEM(state, GetMemoryChunkSpace(state->memtuples));
2650
2651         /*
2652          * Use all the remaining memory we have available for read buffers among
2653          * the input tapes.
2654          *
2655          * We do this only after checking for the case that we produced only one
2656          * initial run, because there is no need to use a large read buffer when
2657          * we're reading from a single tape.  With one tape, the I/O pattern will
2658          * be the same regardless of the buffer size.
2659          *
2660          * We don't try to "rebalance" the memory among tapes, when we start a new
2661          * merge phase, even if some tapes are inactive in the new phase.  That
2662          * would be hard, because logtape.c doesn't know where one run ends and
2663          * another begins.  When a new merge phase begins, and a tape doesn't
2664          * participate in it, its buffer nevertheless already contains tuples from
2665          * the next run on same tape, so we cannot release the buffer.  That's OK
2666          * in practice, merge performance isn't that sensitive to the amount of
2667          * buffers used, and most merge phases use all or almost all tapes,
2668          * anyway.
2669          */
2670 #ifdef TRACE_SORT
2671         if (trace_sort)
2672                 elog(LOG, "using " INT64_FORMAT " KB of memory for read buffers among %d input tapes",
2673                          (state->availMem) / 1024, numInputTapes);
2674 #endif
2675
2676         state->read_buffer_size = Max(state->availMem / numInputTapes, 0);
2677         USEMEM(state, state->read_buffer_size * numInputTapes);
2678
2679         /* End of step D2: rewind all output tapes to prepare for merging */
2680         for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
2681                 LogicalTapeRewindForRead(state->tapeset, tapenum, state->read_buffer_size);
2682
2683         for (;;)
2684         {
2685                 /*
2686                  * At this point we know that tape[T] is empty.  If there's just one
2687                  * (real or dummy) run left on each input tape, then only one merge
2688                  * pass remains.  If we don't have to produce a materialized sorted
2689                  * tape, we can stop at this point and do the final merge on-the-fly.
2690                  */
2691                 if (!state->randomAccess)
2692                 {
2693                         bool            allOneRun = true;
2694
2695                         Assert(state->tp_runs[state->tapeRange] == 0);
2696                         for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
2697                         {
2698                                 if (state->tp_runs[tapenum] + state->tp_dummy[tapenum] != 1)
2699                                 {
2700                                         allOneRun = false;
2701                                         break;
2702                                 }
2703                         }
2704                         if (allOneRun)
2705                         {
2706                                 /* Tell logtape.c we won't be writing anymore */
2707                                 LogicalTapeSetForgetFreeSpace(state->tapeset);
2708                                 /* Initialize for the final merge pass */
2709                                 beginmerge(state);
2710                                 state->status = TSS_FINALMERGE;
2711                                 return;
2712                         }
2713                 }
2714
2715                 /* Step D5: merge runs onto tape[T] until tape[P] is empty */
2716                 while (state->tp_runs[state->tapeRange - 1] ||
2717                            state->tp_dummy[state->tapeRange - 1])
2718                 {
2719                         bool            allDummy = true;
2720
2721                         for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
2722                         {
2723                                 if (state->tp_dummy[tapenum] == 0)
2724                                 {
2725                                         allDummy = false;
2726                                         break;
2727                                 }
2728                         }
2729
2730                         if (allDummy)
2731                         {
2732                                 state->tp_dummy[state->tapeRange]++;
2733                                 for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
2734                                         state->tp_dummy[tapenum]--;
2735                         }
2736                         else
2737                                 mergeonerun(state);
2738                 }
2739
2740                 /* Step D6: decrease level */
2741                 if (--state->Level == 0)
2742                         break;
2743                 /* rewind output tape T to use as new input */
2744                 LogicalTapeRewindForRead(state->tapeset, state->tp_tapenum[state->tapeRange],
2745                                                                  state->read_buffer_size);
2746                 /* rewind used-up input tape P, and prepare it for write pass */
2747                 LogicalTapeRewindForWrite(state->tapeset, state->tp_tapenum[state->tapeRange - 1]);
2748                 state->tp_runs[state->tapeRange - 1] = 0;
2749
2750                 /*
2751                  * reassign tape units per step D6; note we no longer care about A[]
2752                  */
2753                 svTape = state->tp_tapenum[state->tapeRange];
2754                 svDummy = state->tp_dummy[state->tapeRange];
2755                 svRuns = state->tp_runs[state->tapeRange];
2756                 for (tapenum = state->tapeRange; tapenum > 0; tapenum--)
2757                 {
2758                         state->tp_tapenum[tapenum] = state->tp_tapenum[tapenum - 1];
2759                         state->tp_dummy[tapenum] = state->tp_dummy[tapenum - 1];
2760                         state->tp_runs[tapenum] = state->tp_runs[tapenum - 1];
2761                 }
2762                 state->tp_tapenum[0] = svTape;
2763                 state->tp_dummy[0] = svDummy;
2764                 state->tp_runs[0] = svRuns;
2765         }
2766
2767         /*
2768          * Done.  Knuth says that the result is on TAPE[1], but since we exited
2769          * the loop without performing the last iteration of step D6, we have not
2770          * rearranged the tape unit assignment, and therefore the result is on
2771          * TAPE[T].  We need to do it this way so that we can freeze the final
2772          * output tape while rewinding it.  The last iteration of step D6 would be
2773          * a waste of cycles anyway...
2774          */
2775         state->result_tape = state->tp_tapenum[state->tapeRange];
2776         LogicalTapeFreeze(state->tapeset, state->result_tape);
2777         state->status = TSS_SORTEDONTAPE;
2778
2779         /* Release the read buffers of all the other tapes, by rewinding them. */
2780         for (tapenum = 0; tapenum < state->maxTapes; tapenum++)
2781         {
2782                 if (tapenum != state->result_tape)
2783                         LogicalTapeRewindForWrite(state->tapeset, tapenum);
2784         }
2785 }
2786
2787 /*
2788  * Merge one run from each input tape, except ones with dummy runs.
2789  *
2790  * This is the inner loop of Algorithm D step D5.  We know that the
2791  * output tape is TAPE[T].
2792  */
2793 static void
2794 mergeonerun(Tuplesortstate *state)
2795 {
2796         int                     destTape = state->tp_tapenum[state->tapeRange];
2797         int                     srcTape;
2798
2799         /*
2800          * Start the merge by loading one tuple from each active source tape into
2801          * the heap.  We can also decrease the input run/dummy run counts.
2802          */
2803         beginmerge(state);
2804
2805         /*
2806          * Execute merge by repeatedly extracting lowest tuple in heap, writing it
2807          * out, and replacing it with next tuple from same tape (if there is
2808          * another one).
2809          */
2810         while (state->memtupcount > 0)
2811         {
2812                 SortTuple       stup;
2813
2814                 /* write the tuple to destTape */
2815                 srcTape = state->memtuples[0].tupindex;
2816                 WRITETUP(state, destTape, &state->memtuples[0]);
2817
2818                 /* recycle the slot of the tuple we just wrote out, for the next read */
2819                 if (state->memtuples[0].tuple)
2820                         RELEASE_SLAB_SLOT(state, state->memtuples[0].tuple);
2821
2822                 /*
2823                  * pull next tuple from the tape, and replace the written-out tuple in
2824                  * the heap with it.
2825                  */
2826                 if (mergereadnext(state, srcTape, &stup))
2827                 {
2828                         stup.tupindex = srcTape;
2829                         tuplesort_heap_replace_top(state, &stup, false);
2830
2831                 }
2832                 else
2833                         tuplesort_heap_delete_top(state, false);
2834         }
2835
2836         /*
2837          * When the heap empties, we're done.  Write an end-of-run marker on the
2838          * output tape, and increment its count of real runs.
2839          */
2840         markrunend(state, destTape);
2841         state->tp_runs[state->tapeRange]++;
2842
2843 #ifdef TRACE_SORT
2844         if (trace_sort)
2845                 elog(LOG, "finished %d-way merge step: %s", state->activeTapes,
2846                          pg_rusage_show(&state->ru_start));
2847 #endif
2848 }
2849
2850 /*
2851  * beginmerge - initialize for a merge pass
2852  *
2853  * We decrease the counts of real and dummy runs for each tape, and mark
2854  * which tapes contain active input runs in mergeactive[].  Then, fill the
2855  * merge heap with the first tuple from each active tape.
2856  */
2857 static void
2858 beginmerge(Tuplesortstate *state)
2859 {
2860         int                     activeTapes;
2861         int                     tapenum;
2862         int                     srcTape;
2863
2864         /* Heap should be empty here */
2865         Assert(state->memtupcount == 0);
2866
2867         /* Adjust run counts and mark the active tapes */
2868         memset(state->mergeactive, 0,
2869                    state->maxTapes * sizeof(*state->mergeactive));
2870         activeTapes = 0;
2871         for (tapenum = 0; tapenum < state->tapeRange; tapenum++)
2872         {
2873                 if (state->tp_dummy[tapenum] > 0)
2874                         state->tp_dummy[tapenum]--;
2875                 else
2876                 {
2877                         Assert(state->tp_runs[tapenum] > 0);
2878                         state->tp_runs[tapenum]--;
2879                         srcTape = state->tp_tapenum[tapenum];
2880                         state->mergeactive[srcTape] = true;
2881                         activeTapes++;
2882                 }
2883         }
2884         Assert(activeTapes > 0);
2885         state->activeTapes = activeTapes;
2886
2887         /* Load the merge heap with the first tuple from each input tape */
2888         for (srcTape = 0; srcTape < state->maxTapes; srcTape++)
2889         {
2890                 SortTuple       tup;
2891
2892                 if (mergereadnext(state, srcTape, &tup))
2893                 {
2894                         tup.tupindex = srcTape;
2895                         tuplesort_heap_insert(state, &tup, false);
2896                 }
2897         }
2898 }
2899
2900 /*
2901  * mergereadnext - read next tuple from one merge input tape
2902  *
2903  * Returns false on EOF.
2904  */
2905 static bool
2906 mergereadnext(Tuplesortstate *state, int srcTape, SortTuple *stup)
2907 {
2908         unsigned int tuplen;
2909
2910         if (!state->mergeactive[srcTape])
2911                 return false;                   /* tape's run is already exhausted */
2912
2913         /* read next tuple, if any */
2914         if ((tuplen = getlen(state, srcTape, true)) == 0)
2915         {
2916                 state->mergeactive[srcTape] = false;
2917                 return false;
2918         }
2919         READTUP(state, stup, srcTape, tuplen);
2920
2921         return true;
2922 }
2923
2924 /*
2925  * dumptuples - remove tuples from memtuples and write to tape
2926  *
2927  * This is used during initial-run building, but not during merging.
2928  *
2929  * When alltuples = false and replacement selection is still active, dump
2930  * only enough tuples to get under the availMem limit (and leave at least
2931  * one tuple in memtuples, since puttuple will then assume it is a heap that
2932  * has a tuple to compare to).  We always insist there be at least one free
2933  * slot in the memtuples[] array.
2934  *
2935  * When alltuples = true, dump everything currently in memory.  (This
2936  * case is only used at end of input data, although in practice only the
2937  * first run could fail to dump all tuples when we LACKMEM(), and only
2938  * when replacement selection is active.)
2939  *
2940  * If, when replacement selection is active, we see that the tuple run
2941  * number at the top of the heap has changed, start a new run.  This must be
2942  * the first run, because replacement selection is always abandoned for all
2943  * further runs.
2944  */
2945 static void
2946 dumptuples(Tuplesortstate *state, bool alltuples)
2947 {
2948         while (alltuples ||
2949                    (LACKMEM(state) && state->memtupcount > 1) ||
2950                    state->memtupcount >= state->memtupsize)
2951         {
2952                 if (state->replaceActive)
2953                 {
2954                         /*
2955                          * Still holding out for a case favorable to replacement
2956                          * selection. Still incrementally spilling using heap.
2957                          *
2958                          * Dump the heap's frontmost entry, and remove it from the heap.
2959                          */
2960                         Assert(state->memtupcount > 0);
2961                         WRITETUP(state, state->tp_tapenum[state->destTape],
2962                                          &state->memtuples[0]);
2963                         tuplesort_heap_delete_top(state, true);
2964                 }
2965                 else
2966                 {
2967                         /*
2968                          * Once committed to quicksorting runs, never incrementally spill
2969                          */
2970                         dumpbatch(state, alltuples);
2971                         break;
2972                 }
2973
2974                 /*
2975                  * If top run number has changed, we've finished the current run (this
2976                  * can only be the first run), and will no longer spill incrementally.
2977                  */
2978                 if (state->memtupcount == 0 ||
2979                         state->memtuples[0].tupindex == HEAP_RUN_NEXT)
2980                 {
2981                         markrunend(state, state->tp_tapenum[state->destTape]);
2982                         Assert(state->currentRun == RUN_FIRST);
2983                         state->currentRun++;
2984                         state->tp_runs[state->destTape]++;
2985                         state->tp_dummy[state->destTape]--; /* per Alg D step D2 */
2986
2987 #ifdef TRACE_SORT
2988                         if (trace_sort)
2989                                 elog(LOG, "finished incrementally writing %s run %d to tape %d: %s",
2990                                          (state->memtupcount == 0) ? "only" : "first",
2991                                          state->currentRun, state->destTape,
2992                                          pg_rusage_show(&state->ru_start));
2993 #endif
2994
2995                         /*
2996                          * Done if heap is empty, which is possible when there is only one
2997                          * long run.
2998                          */
2999                         Assert(state->currentRun == RUN_SECOND);
3000                         if (state->memtupcount == 0)
3001                         {
3002                                 /*
3003                                  * Replacement selection best case; no final merge required,
3004                                  * because there was only one initial run (second run has no
3005                                  * tuples).  See RUN_SECOND case in mergeruns().
3006                                  */
3007                                 break;
3008                         }
3009
3010                         /*
3011                          * Abandon replacement selection for second run (as well as any
3012                          * subsequent runs).
3013                          */
3014                         state->replaceActive = false;
3015
3016                         /*
3017                          * First tuple of next run should not be heapified, and so will
3018                          * bear placeholder run number.  In practice this must actually be
3019                          * the second run, which just became the currentRun, so we're
3020                          * clear to quicksort and dump the tuples in batch next time
3021                          * memtuples becomes full.
3022                          */
3023                         Assert(state->memtuples[0].tupindex == HEAP_RUN_NEXT);
3024                         selectnewtape(state);
3025                 }
3026         }
3027 }
3028
3029 /*
3030  * dumpbatch - sort and dump all memtuples, forming one run on tape
3031  *
3032  * Second or subsequent runs are never heapified by this module (although
3033  * heapification still respects run number differences between the first and
3034  * second runs), and a heap (replacement selection priority queue) is often
3035  * avoided in the first place.
3036  */
3037 static void
3038 dumpbatch(Tuplesortstate *state, bool alltuples)
3039 {
3040         int                     memtupwrite;
3041         int                     i;
3042
3043         /*
3044          * Final call might require no sorting, in rare cases where we just so
3045          * happen to have previously LACKMEM()'d at the point where exactly all
3046          * remaining tuples are loaded into memory, just before input was
3047          * exhausted.
3048          *
3049          * In general, short final runs are quite possible.  Rather than allowing
3050          * a special case where there was a superfluous selectnewtape() call (i.e.
3051          * a call with no subsequent run actually written to destTape), we prefer
3052          * to write out a 0 tuple run.
3053          *
3054          * mergereadnext() is prepared for 0 tuple runs, and will reliably mark
3055          * the tape inactive for the merge when called from beginmerge().  This
3056          * case is therefore similar to the case where mergeonerun() finds a dummy
3057          * run for the tape, and so doesn't need to merge a run from the tape (or
3058          * conceptually "merges" the dummy run, if you prefer).  According to
3059          * Knuth, Algorithm D "isn't strictly optimal" in its method of
3060          * distribution and dummy run assignment; this edge case seems very
3061          * unlikely to make that appreciably worse.
3062          */
3063         Assert(state->status == TSS_BUILDRUNS);
3064
3065         /*
3066          * It seems unlikely that this limit will ever be exceeded, but take no
3067          * chances
3068          */
3069         if (state->currentRun == INT_MAX)
3070                 ereport(ERROR,
3071                                 (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
3072                                  errmsg("cannot have more than %d runs for an external sort",
3073                                                 INT_MAX)));
3074
3075         state->currentRun++;
3076
3077 #ifdef TRACE_SORT
3078         if (trace_sort)
3079                 elog(LOG, "starting quicksort of run %d: %s",
3080                          state->currentRun, pg_rusage_show(&state->ru_start));
3081 #endif
3082
3083         /*
3084          * Sort all tuples accumulated within the allowed amount of memory for
3085          * this run using quicksort
3086          */
3087         tuplesort_sort_memtuples(state);
3088
3089 #ifdef TRACE_SORT
3090         if (trace_sort)
3091                 elog(LOG, "finished quicksort of run %d: %s",
3092                          state->currentRun, pg_rusage_show(&state->ru_start));
3093 #endif
3094
3095         memtupwrite = state->memtupcount;
3096         for (i = 0; i < memtupwrite; i++)
3097         {
3098                 WRITETUP(state, state->tp_tapenum[state->destTape],
3099                                  &state->memtuples[i]);
3100                 state->memtupcount--;
3101         }
3102
3103         /*
3104          * Reset tuple memory.  We've freed all of the tuples that we previously
3105          * allocated.  It's important to avoid fragmentation when there is a stark
3106          * change in the sizes of incoming tuples.  Fragmentation due to
3107          * AllocSetFree's bucketing by size class might be particularly bad if
3108          * this step wasn't taken.
3109          */
3110         MemoryContextReset(state->tuplecontext);
3111
3112         markrunend(state, state->tp_tapenum[state->destTape]);
3113         state->tp_runs[state->destTape]++;
3114         state->tp_dummy[state->destTape]--; /* per Alg D step D2 */
3115
3116 #ifdef TRACE_SORT
3117         if (trace_sort)
3118                 elog(LOG, "finished writing run %d to tape %d: %s",
3119                          state->currentRun, state->destTape,
3120                          pg_rusage_show(&state->ru_start));
3121 #endif
3122
3123         if (!alltuples)
3124                 selectnewtape(state);
3125 }
3126
3127 /*
3128  * tuplesort_rescan             - rewind and replay the scan
3129  */
3130 void
3131 tuplesort_rescan(Tuplesortstate *state)
3132 {
3133         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
3134
3135         Assert(state->randomAccess);
3136
3137         switch (state->status)
3138         {
3139                 case TSS_SORTEDINMEM:
3140                         state->current = 0;
3141                         state->eof_reached = false;
3142                         state->markpos_offset = 0;
3143                         state->markpos_eof = false;
3144                         break;
3145                 case TSS_SORTEDONTAPE:
3146                         LogicalTapeRewindForRead(state->tapeset,
3147                                                                          state->result_tape,
3148                                                                          0);
3149                         state->eof_reached = false;
3150                         state->markpos_block = 0L;
3151                         state->markpos_offset = 0;
3152                         state->markpos_eof = false;
3153                         break;
3154                 default:
3155                         elog(ERROR, "invalid tuplesort state");
3156                         break;
3157         }
3158
3159         MemoryContextSwitchTo(oldcontext);
3160 }
3161
3162 /*
3163  * tuplesort_markpos    - saves current position in the merged sort file
3164  */
3165 void
3166 tuplesort_markpos(Tuplesortstate *state)
3167 {
3168         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
3169
3170         Assert(state->randomAccess);
3171
3172         switch (state->status)
3173         {
3174                 case TSS_SORTEDINMEM:
3175                         state->markpos_offset = state->current;
3176                         state->markpos_eof = state->eof_reached;
3177                         break;
3178                 case TSS_SORTEDONTAPE:
3179                         LogicalTapeTell(state->tapeset,
3180                                                         state->result_tape,
3181                                                         &state->markpos_block,
3182                                                         &state->markpos_offset);
3183                         state->markpos_eof = state->eof_reached;
3184                         break;
3185                 default:
3186                         elog(ERROR, "invalid tuplesort state");
3187                         break;
3188         }
3189
3190         MemoryContextSwitchTo(oldcontext);
3191 }
3192
3193 /*
3194  * tuplesort_restorepos - restores current position in merged sort file to
3195  *                                                last saved position
3196  */
3197 void
3198 tuplesort_restorepos(Tuplesortstate *state)
3199 {
3200         MemoryContext oldcontext = MemoryContextSwitchTo(state->sortcontext);
3201
3202         Assert(state->randomAccess);
3203
3204         switch (state->status)
3205         {
3206                 case TSS_SORTEDINMEM:
3207                         state->current = state->markpos_offset;
3208                         state->eof_reached = state->markpos_eof;
3209                         break;
3210                 case TSS_SORTEDONTAPE:
3211                         LogicalTapeSeek(state->tapeset,
3212                                                         state->result_tape,
3213                                                         state->markpos_block,
3214                                                         state->markpos_offset);
3215                         state->eof_reached = state->markpos_eof;
3216                         break;
3217                 default:
3218                         elog(ERROR, "invalid tuplesort state");
3219                         break;
3220         }
3221
3222         MemoryContextSwitchTo(oldcontext);
3223 }
3224
3225 /*
3226  * tuplesort_get_stats - extract summary statistics
3227  *
3228  * This can be called after tuplesort_performsort() finishes to obtain
3229  * printable summary information about how the sort was performed.
3230  * spaceUsed is measured in kilobytes.
3231  */
3232 void
3233 tuplesort_get_stats(Tuplesortstate *state,
3234                                         const char **sortMethod,
3235                                         const char **spaceType,
3236                                         long *spaceUsed)
3237 {
3238         /*
3239          * Note: it might seem we should provide both memory and disk usage for a
3240          * disk-based sort.  However, the current code doesn't track memory space
3241          * accurately once we have begun to return tuples to the caller (since we
3242          * don't account for pfree's the caller is expected to do), so we cannot
3243          * rely on availMem in a disk sort.  This does not seem worth the overhead
3244          * to fix.  Is it worth creating an API for the memory context code to
3245          * tell us how much is actually used in sortcontext?
3246          */
3247         if (state->tapeset)
3248         {
3249                 *spaceType = "Disk";
3250                 *spaceUsed = LogicalTapeSetBlocks(state->tapeset) * (BLCKSZ / 1024);
3251         }
3252         else
3253         {
3254                 *spaceType = "Memory";
3255                 *spaceUsed = (state->allowedMem - state->availMem + 1023) / 1024;
3256         }
3257
3258         switch (state->status)
3259         {
3260                 case TSS_SORTEDINMEM:
3261                         if (state->boundUsed)
3262                                 *sortMethod = "top-N heapsort";
3263                         else
3264                                 *sortMethod = "quicksort";
3265                         break;
3266                 case TSS_SORTEDONTAPE:
3267                         *sortMethod = "external sort";
3268                         break;
3269                 case TSS_FINALMERGE:
3270                         *sortMethod = "external merge";
3271                         break;
3272                 default:
3273                         *sortMethod = "still in progress";
3274                         break;
3275         }
3276 }
3277
3278
3279 /*
3280  * Heap manipulation routines, per Knuth's Algorithm 5.2.3H.
3281  *
3282  * Compare two SortTuples.  If checkIndex is true, use the tuple index
3283  * as the front of the sort key; otherwise, no.
3284  *
3285  * Note that for checkIndex callers, the heap invariant is never
3286  * maintained beyond the first run, and so there are no COMPARETUP()
3287  * calls needed to distinguish tuples in HEAP_RUN_NEXT.
3288  */
3289
3290 #define HEAPCOMPARE(tup1,tup2) \
3291         (checkIndex && ((tup1)->tupindex != (tup2)->tupindex || \
3292                                         (tup1)->tupindex == HEAP_RUN_NEXT) ? \
3293          ((tup1)->tupindex) - ((tup2)->tupindex) : \
3294          COMPARETUP(state, tup1, tup2))
3295
3296 /*
3297  * Convert the existing unordered array of SortTuples to a bounded heap,
3298  * discarding all but the smallest "state->bound" tuples.
3299  *
3300  * When working with a bounded heap, we want to keep the largest entry
3301  * at the root (array entry zero), instead of the smallest as in the normal
3302  * sort case.  This allows us to discard the largest entry cheaply.
3303  * Therefore, we temporarily reverse the sort direction.
3304  *
3305  * We assume that all entries in a bounded heap will always have tupindex
3306  * zero; it therefore doesn't matter that HEAPCOMPARE() doesn't reverse
3307  * the direction of comparison for tupindexes.
3308  */
3309 static void
3310 make_bounded_heap(Tuplesortstate *state)
3311 {
3312         int                     tupcount = state->memtupcount;
3313         int                     i;
3314
3315         Assert(state->status == TSS_INITIAL);
3316         Assert(state->bounded);
3317         Assert(tupcount >= state->bound);
3318
3319         /* Reverse sort direction so largest entry will be at root */
3320         reversedirection(state);
3321
3322         state->memtupcount = 0;         /* make the heap empty */
3323         for (i = 0; i < tupcount; i++)
3324         {
3325                 if (state->memtupcount < state->bound)
3326                 {
3327                         /* Insert next tuple into heap */
3328                         /* Must copy source tuple to avoid possible overwrite */
3329                         SortTuple       stup = state->memtuples[i];
3330
3331                         stup.tupindex = 0;      /* not used */
3332                         tuplesort_heap_insert(state, &stup, false);
3333                 }
3334                 else
3335                 {
3336                         /*
3337                          * The heap is full.  Replace the largest entry with the new
3338                          * tuple, or just discard it, if it's larger than anything already
3339                          * in the heap.
3340                          */
3341                         if (COMPARETUP(state, &state->memtuples[i], &state->memtuples[0]) <= 0)
3342                         {
3343                                 free_sort_tuple(state, &state->memtuples[i]);
3344                                 CHECK_FOR_INTERRUPTS();
3345                         }
3346                         else
3347                                 tuplesort_heap_replace_top(state, &state->memtuples[i], false);
3348                 }
3349         }
3350
3351         Assert(state->memtupcount == state->bound);
3352         state->status = TSS_BOUNDED;
3353 }
3354
3355 /*
3356  * Convert the bounded heap to a properly-sorted array
3357  */
3358 static void
3359 sort_bounded_heap(Tuplesortstate *state)
3360 {
3361         int                     tupcount = state->memtupcount;
3362
3363         Assert(state->status == TSS_BOUNDED);
3364         Assert(state->bounded);
3365         Assert(tupcount == state->bound);
3366
3367         /*
3368          * We can unheapify in place because each delete-top call will remove the
3369          * largest entry, which we can promptly store in the newly freed slot at
3370          * the end.  Once we're down to a single-entry heap, we're done.
3371          */
3372         while (state->memtupcount > 1)
3373         {
3374                 SortTuple       stup = state->memtuples[0];
3375
3376                 /* this sifts-up the next-largest entry and decreases memtupcount */
3377                 tuplesort_heap_delete_top(state, false);
3378                 state->memtuples[state->memtupcount] = stup;
3379         }
3380         state->memtupcount = tupcount;
3381
3382         /*
3383          * Reverse sort direction back to the original state.  This is not
3384          * actually necessary but seems like a good idea for tidiness.
3385          */
3386         reversedirection(state);
3387
3388         state->status = TSS_SORTEDINMEM;
3389         state->boundUsed = true;
3390 }
3391
3392 /*
3393  * Sort all memtuples using specialized qsort() routines.
3394  *
3395  * Quicksort is used for small in-memory sorts.  Quicksort is also generally
3396  * preferred to replacement selection for generating runs during external sort
3397  * operations, although replacement selection is sometimes used for the first
3398  * run.
3399  */
3400 static void
3401 tuplesort_sort_memtuples(Tuplesortstate *state)
3402 {
3403         if (state->memtupcount > 1)
3404         {
3405                 /* Can we use the single-key sort function? */
3406                 if (state->onlyKey != NULL)
3407                         qsort_ssup(state->memtuples, state->memtupcount,
3408                                            state->onlyKey);
3409                 else
3410                         qsort_tuple(state->memtuples,
3411                                                 state->memtupcount,
3412                                                 state->comparetup,
3413                                                 state);
3414         }
3415 }
3416
3417 /*
3418  * Insert a new tuple into an empty or existing heap, maintaining the
3419  * heap invariant.  Caller is responsible for ensuring there's room.
3420  *
3421  * Note: For some callers, tuple points to a memtuples[] entry above the
3422  * end of the heap.  This is safe as long as it's not immediately adjacent
3423  * to the end of the heap (ie, in the [memtupcount] array entry) --- if it
3424  * is, it might get overwritten before being moved into the heap!
3425  */
3426 static void
3427 tuplesort_heap_insert(Tuplesortstate *state, SortTuple *tuple,
3428                                           bool checkIndex)
3429 {
3430         SortTuple  *memtuples;
3431         int                     j;
3432
3433         memtuples = state->memtuples;
3434         Assert(state->memtupcount < state->memtupsize);
3435         Assert(!checkIndex || tuple->tupindex == RUN_FIRST);
3436
3437         CHECK_FOR_INTERRUPTS();
3438
3439         /*
3440          * Sift-up the new entry, per Knuth 5.2.3 exercise 16. Note that Knuth is
3441          * using 1-based array indexes, not 0-based.
3442          */
3443         j = state->memtupcount++;
3444         while (j > 0)
3445         {
3446                 int                     i = (j - 1) >> 1;
3447
3448                 if (HEAPCOMPARE(tuple, &memtuples[i]) >= 0)
3449                         break;
3450                 memtuples[j] = memtuples[i];
3451                 j = i;
3452         }
3453         memtuples[j] = *tuple;
3454 }
3455
3456 /*
3457  * Remove the tuple at state->memtuples[0] from the heap.  Decrement
3458  * memtupcount, and sift up to maintain the heap invariant.
3459  *
3460  * The caller has already free'd the tuple the top node points to,
3461  * if necessary.
3462  */
3463 static void
3464 tuplesort_heap_delete_top(Tuplesortstate *state, bool checkIndex)
3465 {
3466         SortTuple  *memtuples = state->memtuples;
3467         SortTuple  *tuple;
3468
3469         Assert(!checkIndex || state->currentRun == RUN_FIRST);
3470         if (--state->memtupcount <= 0)
3471                 return;
3472
3473         /*
3474          * Remove the last tuple in the heap, and re-insert it, by replacing the
3475          * current top node with it.
3476          */
3477         tuple = &memtuples[state->memtupcount];
3478         tuplesort_heap_replace_top(state, tuple, checkIndex);
3479 }
3480
3481 /*
3482  * Replace the tuple at state->memtuples[0] with a new tuple.  Sift up to
3483  * maintain the heap invariant.
3484  *
3485  * This corresponds to Knuth's "sift-up" algorithm (Algorithm 5.2.3H,
3486  * Heapsort, steps H3-H8).
3487  */
3488 static void
3489 tuplesort_heap_replace_top(Tuplesortstate *state, SortTuple *tuple,
3490                                                    bool checkIndex)
3491 {
3492         SortTuple  *memtuples = state->memtuples;
3493         unsigned int i,
3494                                 n;
3495
3496         Assert(!checkIndex || state->currentRun == RUN_FIRST);
3497         Assert(state->memtupcount >= 1);
3498
3499         CHECK_FOR_INTERRUPTS();
3500
3501         /*
3502          * state->memtupcount is "int", but we use "unsigned int" for i, j, n.
3503          * This prevents overflow in the "2 * i + 1" calculation, since at the top
3504          * of the loop we must have i < n <= INT_MAX <= UINT_MAX/2.
3505          */
3506         n = state->memtupcount;
3507         i = 0;                                          /* i is where the "hole" is */
3508         for (;;)
3509         {
3510                 unsigned int j = 2 * i + 1;
3511
3512                 if (j >= n)
3513                         break;
3514                 if (j + 1 < n &&
3515                         HEAPCOMPARE(&memtuples[j], &memtuples[j + 1]) > 0)
3516                         j++;
3517                 if (HEAPCOMPARE(tuple, &memtuples[j]) <= 0)
3518                         break;
3519                 memtuples[i] = memtuples[j];
3520                 i = j;
3521         }
3522         memtuples[i] = *tuple;
3523 }
3524
3525 /*
3526  * Function to reverse the sort direction from its current state
3527  *
3528  * It is not safe to call this when performing hash tuplesorts
3529  */
3530 static void
3531 reversedirection(Tuplesortstate *state)
3532 {
3533         SortSupport sortKey = state->sortKeys;
3534         int                     nkey;
3535
3536         for (nkey = 0; nkey < state->nKeys; nkey++, sortKey++)
3537         {
3538                 sortKey->ssup_reverse = !sortKey->ssup_reverse;
3539                 sortKey->ssup_nulls_first = !sortKey->ssup_nulls_first;
3540         }
3541 }
3542
3543
3544 /*
3545  * Tape interface routines
3546  */
3547
3548 static unsigned int
3549 getlen(Tuplesortstate *state, int tapenum, bool eofOK)
3550 {
3551         unsigned int len;
3552
3553         if (LogicalTapeRead(state->tapeset, tapenum,
3554                                                 &len, sizeof(len)) != sizeof(len))
3555                 elog(ERROR, "unexpected end of tape");
3556         if (len == 0 && !eofOK)
3557                 elog(ERROR, "unexpected end of data");
3558         return len;
3559 }
3560
3561 static void
3562 markrunend(Tuplesortstate *state, int tapenum)
3563 {
3564         unsigned int len = 0;
3565
3566         LogicalTapeWrite(state->tapeset, tapenum, (void *) &len, sizeof(len));
3567 }
3568
3569 /*
3570  * Get memory for tuple from within READTUP() routine.
3571  *
3572  * We use next free slot from the slab allocator, or palloc() if the tuple
3573  * is too large for that.
3574  */
3575 static void *
3576 readtup_alloc(Tuplesortstate *state, Size tuplen)
3577 {
3578         SlabSlot   *buf;
3579
3580         /*
3581          * We pre-allocate enough slots in the slab arena that we should never run
3582          * out.
3583          */
3584         Assert(state->slabFreeHead);
3585
3586         if (tuplen > SLAB_SLOT_SIZE || !state->slabFreeHead)
3587                 return MemoryContextAlloc(state->sortcontext, tuplen);
3588         else
3589         {
3590                 buf = state->slabFreeHead;
3591                 /* Reuse this slot */
3592                 state->slabFreeHead = buf->nextfree;
3593
3594                 return buf;
3595         }
3596 }
3597
3598
3599 /*
3600  * Routines specialized for HeapTuple (actually MinimalTuple) case
3601  */
3602
3603 static int
3604 comparetup_heap(const SortTuple *a, const SortTuple *b, Tuplesortstate *state)
3605 {
3606         SortSupport sortKey = state->sortKeys;
3607         HeapTupleData ltup;
3608         HeapTupleData rtup;
3609         TupleDesc       tupDesc;
3610         int                     nkey;
3611         int32           compare;
3612         AttrNumber      attno;
3613         Datum           datum1,
3614                                 datum2;
3615         bool            isnull1,
3616                                 isnull2;
3617
3618
3619         /* Compare the leading sort key */
3620         compare = ApplySortComparator(a->datum1, a->isnull1,
3621                                                                   b->datum1, b->isnull1,
3622                                                                   sortKey);
3623         if (compare != 0)
3624                 return compare;
3625
3626         /* Compare additional sort keys */
3627         ltup.t_len = ((MinimalTuple) a->tuple)->t_len + MINIMAL_TUPLE_OFFSET;
3628         ltup.t_data = (HeapTupleHeader) ((char *) a->tuple - MINIMAL_TUPLE_OFFSET);
3629         rtup.t_len = ((MinimalTuple) b->tuple)->t_len + MINIMAL_TUPLE_OFFSET;
3630         rtup.t_data = (HeapTupleHeader) ((char *) b->tuple - MINIMAL_TUPLE_OFFSET);
3631         tupDesc = state->tupDesc;
3632
3633         if (sortKey->abbrev_converter)
3634         {
3635                 attno = sortKey->ssup_attno;
3636
3637                 datum1 = heap_getattr(&ltup, attno, tupDesc, &isnull1);
3638                 datum2 = heap_getattr(&rtup, attno, tupDesc, &isnull2);
3639
3640                 compare = ApplySortAbbrevFullComparator(datum1, isnull1,
3641                                                                                                 datum2, isnull2,
3642                                                                                                 sortKey);
3643                 if (compare != 0)
3644                         return compare;
3645         }
3646
3647         sortKey++;
3648         for (nkey = 1; nkey < state->nKeys; nkey++, sortKey++)
3649         {
3650                 attno = sortKey->ssup_attno;
3651
3652                 datum1 = heap_getattr(&ltup, attno, tupDesc, &isnull1);
3653                 datum2 = heap_getattr(&rtup, attno, tupDesc, &isnull2);
3654
3655                 compare = ApplySortComparator(datum1, isnull1,
3656                                                                           datum2, isnull2,
3657                                                                           sortKey);
3658                 if (compare != 0)
3659                         return compare;
3660         }
3661
3662         return 0;
3663 }
3664
3665 static void
3666 copytup_heap(Tuplesortstate *state, SortTuple *stup, void *tup)
3667 {
3668         /*
3669          * We expect the passed "tup" to be a TupleTableSlot, and form a
3670          * MinimalTuple using the exported interface for that.
3671          */
3672         TupleTableSlot *slot = (TupleTableSlot *) tup;
3673         Datum           original;
3674         MinimalTuple tuple;
3675         HeapTupleData htup;
3676         MemoryContext oldcontext = MemoryContextSwitchTo(state->tuplecontext);
3677
3678         /* copy the tuple into sort storage */
3679         tuple = ExecCopySlotMinimalTuple(slot);
3680         stup->tuple = (void *) tuple;
3681         USEMEM(state, GetMemoryChunkSpace(tuple));
3682         /* set up first-column key value */
3683         htup.t_len = tuple->t_len + MINIMAL_TUPLE_OFFSET;
3684         htup.t_data = (HeapTupleHeader) ((char *) tuple - MINIMAL_TUPLE_OFFSET);
3685         original = heap_getattr(&htup,
3686                                                         state->sortKeys[0].ssup_attno,
3687                                                         state->tupDesc,
3688                                                         &stup->isnull1);
3689
3690         MemoryContextSwitchTo(oldcontext);
3691
3692         if (!state->sortKeys->abbrev_converter || stup->isnull1)
3693         {
3694                 /*
3695                  * Store ordinary Datum representation, or NULL value.  If there is a
3696                  * converter it won't expect NULL values, and cost model is not
3697                  * required to account for NULL, so in that case we avoid calling
3698                  * converter and just set datum1 to zeroed representation (to be
3699                  * consistent, and to support cheap inequality tests for NULL
3700                  * abbreviated keys).
3701                  */
3702                 stup->datum1 = original;
3703         }
3704         else if (!consider_abort_common(state))
3705         {
3706                 /* Store abbreviated key representation */
3707                 stup->datum1 = state->sortKeys->abbrev_converter(original,
3708                                                                                                                  state->sortKeys);
3709         }
3710         else
3711         {
3712                 /* Abort abbreviation */
3713                 int                     i;
3714
3715                 stup->datum1 = original;
3716
3717                 /*
3718                  * Set state to be consistent with never trying abbreviation.
3719                  *
3720                  * Alter datum1 representation in already-copied tuples, so as to
3721                  * ensure a consistent representation (current tuple was just
3722                  * handled).  It does not matter if some dumped tuples are already
3723                  * sorted on tape, since serialized tuples lack abbreviated keys
3724                  * (TSS_BUILDRUNS state prevents control reaching here in any case).
3725                  */
3726                 for (i = 0; i < state->memtupcount; i++)
3727                 {
3728                         SortTuple  *mtup = &state->memtuples[i];
3729
3730                         htup.t_len = ((MinimalTuple) mtup->tuple)->t_len +
3731                                 MINIMAL_TUPLE_OFFSET;
3732                         htup.t_data = (HeapTupleHeader) ((char *) mtup->tuple -
3733                                                                                          MINIMAL_TUPLE_OFFSET);
3734
3735                         mtup->datum1 = heap_getattr(&htup,
3736                                                                                 state->sortKeys[0].ssup_attno,
3737                                                                                 state->tupDesc,
3738                                                                                 &mtup->isnull1);
3739                 }
3740         }
3741 }
3742
3743 static void
3744 writetup_heap(Tuplesortstate *state, int tapenum, SortTuple *stup)
3745 {
3746         MinimalTuple tuple = (MinimalTuple) stup->tuple;
3747
3748         /* the part of the MinimalTuple we'll write: */
3749         char       *tupbody = (char *) tuple + MINIMAL_TUPLE_DATA_OFFSET;
3750         unsigned int tupbodylen = tuple->t_len - MINIMAL_TUPLE_DATA_OFFSET;
3751
3752         /* total on-disk footprint: */
3753         unsigned int tuplen = tupbodylen + sizeof(int);
3754
3755         LogicalTapeWrite(state->tapeset, tapenum,
3756                                          (void *) &tuplen, sizeof(tuplen));
3757         LogicalTapeWrite(state->tapeset, tapenum,
3758                                          (void *) tupbody, tupbodylen);
3759         if (state->randomAccess)        /* need trailing length word? */
3760                 LogicalTapeWrite(state->tapeset, tapenum,
3761                                                  (void *) &tuplen, sizeof(tuplen));
3762
3763         if (!state->slabAllocatorUsed)
3764         {
3765                 FREEMEM(state, GetMemoryChunkSpace(tuple));
3766                 heap_free_minimal_tuple(tuple);
3767         }
3768 }
3769
3770 static void
3771 readtup_heap(Tuplesortstate *state, SortTuple *stup,
3772                          int tapenum, unsigned int len)
3773 {
3774         unsigned int tupbodylen = len - sizeof(int);
3775         unsigned int tuplen = tupbodylen + MINIMAL_TUPLE_DATA_OFFSET;
3776         MinimalTuple tuple = (MinimalTuple) readtup_alloc(state, tuplen);
3777         char       *tupbody = (char *) tuple + MINIMAL_TUPLE_DATA_OFFSET;
3778         HeapTupleData htup;
3779
3780         /* read in the tuple proper */
3781         tuple->t_len = tuplen;
3782         LogicalTapeReadExact(state->tapeset, tapenum,
3783                                                  tupbody, tupbodylen);
3784         if (state->randomAccess)        /* need trailing length word? */
3785                 LogicalTapeReadExact(state->tapeset, tapenum,
3786                                                          &tuplen, sizeof(tuplen));
3787         stup->tuple = (void *) tuple;
3788         /* set up first-column key value */
3789         htup.t_len = tuple->t_len + MINIMAL_TUPLE_OFFSET;
3790         htup.t_data = (HeapTupleHeader) ((char *) tuple - MINIMAL_TUPLE_OFFSET);
3791         stup->datum1 = heap_getattr(&htup,
3792                                                                 state->sortKeys[0].ssup_attno,
3793                                                                 state->tupDesc,
3794                                                                 &stup->isnull1);
3795 }
3796
3797 /*
3798  * Routines specialized for the CLUSTER case (HeapTuple data, with
3799  * comparisons per a btree index definition)
3800  */
3801
3802 static int
3803 comparetup_cluster(const SortTuple *a, const SortTuple *b,
3804                                    Tuplesortstate *state)
3805 {
3806         SortSupport sortKey = state->sortKeys;
3807         HeapTuple       ltup;
3808         HeapTuple       rtup;
3809         TupleDesc       tupDesc;
3810         int                     nkey;
3811         int32           compare;
3812         Datum           datum1,
3813                                 datum2;
3814         bool            isnull1,
3815                                 isnull2;
3816         AttrNumber      leading = state->indexInfo->ii_KeyAttrNumbers[0];
3817
3818         /* Be prepared to compare additional sort keys */
3819         ltup = (HeapTuple) a->tuple;
3820         rtup = (HeapTuple) b->tuple;
3821         tupDesc = state->tupDesc;
3822
3823         /* Compare the leading sort key, if it's simple */
3824         if (leading != 0)
3825         {
3826                 compare = ApplySortComparator(a->datum1, a->isnull1,
3827                                                                           b->datum1, b->isnull1,
3828                                                                           sortKey);
3829                 if (compare != 0)
3830                         return compare;
3831
3832                 if (sortKey->abbrev_converter)
3833                 {
3834                         datum1 = heap_getattr(ltup, leading, tupDesc, &isnull1);
3835                         datum2 = heap_getattr(rtup, leading, tupDesc, &isnull2);
3836
3837                         compare = ApplySortAbbrevFullComparator(datum1, isnull1,
3838                                                                                                         datum2, isnull2,
3839                                                                                                         sortKey);
3840                 }
3841                 if (compare != 0 || state->nKeys == 1)
3842                         return compare;
3843                 /* Compare additional columns the hard way */
3844                 sortKey++;
3845                 nkey = 1;
3846         }
3847         else
3848         {
3849                 /* Must compare all keys the hard way */
3850                 nkey = 0;
3851         }
3852
3853         if (state->indexInfo->ii_Expressions == NULL)
3854         {
3855                 /* If not expression index, just compare the proper heap attrs */
3856
3857                 for (; nkey < state->nKeys; nkey++, sortKey++)
3858                 {
3859                         AttrNumber      attno = state->indexInfo->ii_KeyAttrNumbers[nkey];
3860
3861                         datum1 = heap_getattr(ltup, attno, tupDesc, &isnull1);
3862                         datum2 = heap_getattr(rtup, attno, tupDesc, &isnull2);
3863
3864                         compare = ApplySortComparator(datum1, isnull1,
3865                                                                                   datum2, isnull2,
3866                                                                                   sortKey);
3867                         if (compare != 0)
3868                                 return compare;
3869                 }
3870         }
3871         else
3872         {
3873                 /*
3874                  * In the expression index case, compute the whole index tuple and
3875                  * then compare values.  It would perhaps be faster to compute only as
3876                  * many columns as we need to compare, but that would require
3877                  * duplicating all the logic in FormIndexDatum.
3878                  */
3879                 Datum           l_index_values[INDEX_MAX_KEYS];
3880                 bool            l_index_isnull[INDEX_MAX_KEYS];
3881                 Datum           r_index_values[INDEX_MAX_KEYS];
3882                 bool            r_index_isnull[INDEX_MAX_KEYS];
3883                 TupleTableSlot *ecxt_scantuple;
3884
3885                 /* Reset context each time to prevent memory leakage */
3886                 ResetPerTupleExprContext(state->estate);
3887
3888                 ecxt_scantuple = GetPerTupleExprContext(state->estate)->ecxt_scantuple;
3889
3890                 ExecStoreTuple(ltup, ecxt_scantuple, InvalidBuffer, false);
3891                 FormIndexDatum(state->indexInfo, ecxt_scantuple, state->estate,
3892                                            l_index_values, l_index_isnull);
3893
3894                 ExecStoreTuple(rtup, ecxt_scantuple, InvalidBuffer, false);
3895                 FormIndexDatum(state->indexInfo, ecxt_scantuple, state->estate,
3896                                            r_index_values, r_index_isnull);
3897
3898                 for (; nkey < state->nKeys; nkey++, sortKey++)
3899                 {
3900                         compare = ApplySortComparator(l_index_values[nkey],
3901                                                                                   l_index_isnull[nkey],
3902                                                                                   r_index_values[nkey],
3903                                                                                   r_index_isnull[nkey],
3904                                                                                   sortKey);
3905                         if (compare != 0)
3906                                 return compare;
3907                 }
3908         }
3909
3910         return 0;
3911 }
3912
3913 static void
3914 copytup_cluster(Tuplesortstate *state, SortTuple *stup, void *tup)
3915 {
3916         HeapTuple       tuple = (HeapTuple) tup;
3917         Datum           original;
3918         MemoryContext oldcontext = MemoryContextSwitchTo(state->tuplecontext);
3919
3920         /* copy the tuple into sort storage */
3921         tuple = heap_copytuple(tuple);
3922         stup->tuple = (void *) tuple;
3923         USEMEM(state, GetMemoryChunkSpace(tuple));
3924
3925         MemoryContextSwitchTo(oldcontext);
3926
3927         /*
3928          * set up first-column key value, and potentially abbreviate, if it's a
3929          * simple column
3930          */
3931         if (state->indexInfo->ii_KeyAttrNumbers[0] == 0)
3932                 return;
3933
3934         original = heap_getattr(tuple,
3935                                                         state->indexInfo->ii_KeyAttrNumbers[0],
3936                                                         state->tupDesc,
3937                                                         &stup->isnull1);
3938
3939         if (!state->sortKeys->abbrev_converter || stup->isnull1)
3940         {
3941                 /*
3942                  * Store ordinary Datum representation, or NULL value.  If there is a
3943                  * converter it won't expect NULL values, and cost model is not
3944                  * required to account for NULL, so in that case we avoid calling
3945                  * converter and just set datum1 to zeroed representation (to be
3946                  * consistent, and to support cheap inequality tests for NULL
3947                  * abbreviated keys).
3948                  */
3949                 stup->datum1 = original;
3950         }
3951         else if (!consider_abort_common(state))
3952         {
3953                 /* Store abbreviated key representation */
3954                 stup->datum1 = state->sortKeys->abbrev_converter(original,
3955                                                                                                                  state->sortKeys);
3956         }
3957         else
3958         {
3959                 /* Abort abbreviation */
3960                 int                     i;
3961
3962                 stup->datum1 = original;
3963
3964                 /*
3965                  * Set state to be consistent with never trying abbreviation.
3966                  *
3967                  * Alter datum1 representation in already-copied tuples, so as to
3968                  * ensure a consistent representation (current tuple was just
3969                  * handled).  It does not matter if some dumped tuples are already
3970                  * sorted on tape, since serialized tuples lack abbreviated keys
3971                  * (TSS_BUILDRUNS state prevents control reaching here in any case).
3972                  */
3973                 for (i = 0; i < state->memtupcount; i++)
3974                 {
3975                         SortTuple  *mtup = &state->memtuples[i];
3976
3977                         tuple = (HeapTuple) mtup->tuple;
3978                         mtup->datum1 = heap_getattr(tuple,
3979                                                                                 state->indexInfo->ii_KeyAttrNumbers[0],
3980                                                                                 state->tupDesc,
3981                                                                                 &mtup->isnull1);
3982                 }
3983         }
3984 }
3985
3986 static void
3987 writetup_cluster(Tuplesortstate *state, int tapenum, SortTuple *stup)
3988 {
3989         HeapTuple       tuple = (HeapTuple) stup->tuple;
3990         unsigned int tuplen = tuple->t_len + sizeof(ItemPointerData) + sizeof(int);
3991
3992         /* We need to store t_self, but not other fields of HeapTupleData */
3993         LogicalTapeWrite(state->tapeset, tapenum,
3994                                          &tuplen, sizeof(tuplen));
3995         LogicalTapeWrite(state->tapeset, tapenum,
3996                                          &tuple->t_self, sizeof(ItemPointerData));
3997         LogicalTapeWrite(state->tapeset, tapenum,
3998                                          tuple->t_data, tuple->t_len);
3999         if (state->randomAccess)        /* need trailing length word? */
4000                 LogicalTapeWrite(state->tapeset, tapenum,
4001                                                  &tuplen, sizeof(tuplen));
4002
4003         if (!state->slabAllocatorUsed)
4004         {
4005                 FREEMEM(state, GetMemoryChunkSpace(tuple));
4006                 heap_freetuple(tuple);
4007         }
4008 }
4009
4010 static void
4011 readtup_cluster(Tuplesortstate *state, SortTuple *stup,
4012                                 int tapenum, unsigned int tuplen)
4013 {
4014         unsigned int t_len = tuplen - sizeof(ItemPointerData) - sizeof(int);
4015         HeapTuple       tuple = (HeapTuple) readtup_alloc(state,
4016                                                                                                   t_len + HEAPTUPLESIZE);
4017
4018         /* Reconstruct the HeapTupleData header */
4019         tuple->t_data = (HeapTupleHeader) ((char *) tuple + HEAPTUPLESIZE);
4020         tuple->t_len = t_len;
4021         LogicalTapeReadExact(state->tapeset, tapenum,
4022                                                  &tuple->t_self, sizeof(ItemPointerData));
4023         /* We don't currently bother to reconstruct t_tableOid */
4024         tuple->t_tableOid = InvalidOid;
4025         /* Read in the tuple body */
4026         LogicalTapeReadExact(state->tapeset, tapenum,
4027                                                  tuple->t_data, tuple->t_len);
4028         if (state->randomAccess)        /* need trailing length word? */
4029                 LogicalTapeReadExact(state->tapeset, tapenum,
4030                                                          &tuplen, sizeof(tuplen));
4031         stup->tuple = (void *) tuple;
4032         /* set up first-column key value, if it's a simple column */
4033         if (state->indexInfo->ii_KeyAttrNumbers[0] != 0)
4034                 stup->datum1 = heap_getattr(tuple,
4035                                                                         state->indexInfo->ii_KeyAttrNumbers[0],
4036                                                                         state->tupDesc,
4037                                                                         &stup->isnull1);
4038 }
4039
4040 /*
4041  * Routines specialized for IndexTuple case
4042  *
4043  * The btree and hash cases require separate comparison functions, but the
4044  * IndexTuple representation is the same so the copy/write/read support
4045  * functions can be shared.
4046  */
4047
4048 static int
4049 comparetup_index_btree(const SortTuple *a, const SortTuple *b,
4050                                            Tuplesortstate *state)
4051 {
4052         /*
4053          * This is similar to comparetup_heap(), but expects index tuples.  There
4054          * is also special handling for enforcing uniqueness, and special
4055          * treatment for equal keys at the end.
4056          */
4057         SortSupport sortKey = state->sortKeys;
4058         IndexTuple      tuple1;
4059         IndexTuple      tuple2;
4060         int                     keysz;
4061         TupleDesc       tupDes;
4062         bool            equal_hasnull = false;
4063         int                     nkey;
4064         int32           compare;
4065         Datum           datum1,
4066                                 datum2;
4067         bool            isnull1,
4068                                 isnull2;
4069
4070
4071         /* Compare the leading sort key */
4072         compare = ApplySortComparator(a->datum1, a->isnull1,
4073                                                                   b->datum1, b->isnull1,
4074                                                                   sortKey);
4075         if (compare != 0)
4076                 return compare;
4077
4078         /* Compare additional sort keys */
4079         tuple1 = (IndexTuple) a->tuple;
4080         tuple2 = (IndexTuple) b->tuple;
4081         keysz = state->nKeys;
4082         tupDes = RelationGetDescr(state->indexRel);
4083
4084         if (sortKey->abbrev_converter)
4085         {
4086                 datum1 = index_getattr(tuple1, 1, tupDes, &isnull1);
4087                 datum2 = index_getattr(tuple2, 1, tupDes, &isnull2);
4088
4089                 compare = ApplySortAbbrevFullComparator(datum1, isnull1,
4090                                                                                                 datum2, isnull2,
4091                                                                                                 sortKey);
4092                 if (compare != 0)
4093                         return compare;
4094         }
4095
4096         /* they are equal, so we only need to examine one null flag */
4097         if (a->isnull1)
4098                 equal_hasnull = true;
4099
4100         sortKey++;
4101         for (nkey = 2; nkey <= keysz; nkey++, sortKey++)
4102         {
4103                 datum1 = index_getattr(tuple1, nkey, tupDes, &isnull1);
4104                 datum2 = index_getattr(tuple2, nkey, tupDes, &isnull2);
4105
4106                 compare = ApplySortComparator(datum1, isnull1,
4107                                                                           datum2, isnull2,
4108                                                                           sortKey);
4109                 if (compare != 0)
4110                         return compare;         /* done when we find unequal attributes */
4111
4112                 /* they are equal, so we only need to examine one null flag */
4113                 if (isnull1)
4114                         equal_hasnull = true;
4115         }
4116
4117         /*
4118          * If btree has asked us to enforce uniqueness, complain if two equal
4119          * tuples are detected (unless there was at least one NULL field).
4120          *
4121          * It is sufficient to make the test here, because if two tuples are equal
4122          * they *must* get compared at some stage of the sort --- otherwise the
4123          * sort algorithm wouldn't have checked whether one must appear before the
4124          * other.
4125          */
4126         if (state->enforceUnique && !equal_hasnull)
4127         {
4128                 Datum           values[INDEX_MAX_KEYS];
4129                 bool            isnull[INDEX_MAX_KEYS];
4130                 char       *key_desc;
4131
4132                 /*
4133                  * Some rather brain-dead implementations of qsort (such as the one in
4134                  * QNX 4) will sometimes call the comparison routine to compare a
4135                  * value to itself, but we always use our own implementation, which
4136                  * does not.
4137                  */
4138                 Assert(tuple1 != tuple2);
4139
4140                 index_deform_tuple(tuple1, tupDes, values, isnull);
4141
4142                 key_desc = BuildIndexValueDescription(state->indexRel, values, isnull);
4143
4144                 ereport(ERROR,
4145                                 (errcode(ERRCODE_UNIQUE_VIOLATION),
4146                                  errmsg("could not create unique index \"%s\"",
4147                                                 RelationGetRelationName(state->indexRel)),
4148                                  key_desc ? errdetail("Key %s is duplicated.", key_desc) :
4149                                  errdetail("Duplicate keys exist."),
4150                                  errtableconstraint(state->heapRel,
4151                                                                         RelationGetRelationName(state->indexRel))));
4152         }
4153
4154         /*
4155          * If key values are equal, we sort on ItemPointer.  This does not affect
4156          * validity of the finished index, but it may be useful to have index
4157          * scans in physical order.
4158          */
4159         {
4160                 BlockNumber blk1 = ItemPointerGetBlockNumber(&tuple1->t_tid);
4161                 BlockNumber blk2 = ItemPointerGetBlockNumber(&tuple2->t_tid);
4162
4163                 if (blk1 != blk2)
4164                         return (blk1 < blk2) ? -1 : 1;
4165         }
4166         {
4167                 OffsetNumber pos1 = ItemPointerGetOffsetNumber(&tuple1->t_tid);
4168                 OffsetNumber pos2 = ItemPointerGetOffsetNumber(&tuple2->t_tid);
4169
4170                 if (pos1 != pos2)
4171                         return (pos1 < pos2) ? -1 : 1;
4172         }
4173
4174         return 0;
4175 }
4176
4177 static int
4178 comparetup_index_hash(const SortTuple *a, const SortTuple *b,
4179                                           Tuplesortstate *state)
4180 {
4181         Bucket          bucket1;
4182         Bucket          bucket2;
4183         IndexTuple      tuple1;
4184         IndexTuple      tuple2;
4185
4186         /*
4187          * Fetch hash keys and mask off bits we don't want to sort by. We know
4188          * that the first column of the index tuple is the hash key.
4189          */
4190         Assert(!a->isnull1);
4191         bucket1 = _hash_hashkey2bucket(DatumGetUInt32(a->datum1),
4192                                                                    state->max_buckets, state->high_mask,
4193                                                                    state->low_mask);
4194         Assert(!b->isnull1);
4195         bucket2 = _hash_hashkey2bucket(DatumGetUInt32(b->datum1),
4196                                                                    state->max_buckets, state->high_mask,
4197                                                                    state->low_mask);
4198         if (bucket1 > bucket2)
4199                 return 1;
4200         else if (bucket1 < bucket2)
4201                 return -1;
4202
4203         /*
4204          * If hash values are equal, we sort on ItemPointer.  This does not affect
4205          * validity of the finished index, but it may be useful to have index
4206          * scans in physical order.
4207          */
4208         tuple1 = (IndexTuple) a->tuple;
4209         tuple2 = (IndexTuple) b->tuple;
4210
4211         {
4212                 BlockNumber blk1 = ItemPointerGetBlockNumber(&tuple1->t_tid);
4213                 BlockNumber blk2 = ItemPointerGetBlockNumber(&tuple2->t_tid);
4214
4215                 if (blk1 != blk2)
4216                         return (blk1 < blk2) ? -1 : 1;
4217         }
4218         {
4219                 OffsetNumber pos1 = ItemPointerGetOffsetNumber(&tuple1->t_tid);
4220                 OffsetNumber pos2 = ItemPointerGetOffsetNumber(&tuple2->t_tid);
4221
4222                 if (pos1 != pos2)
4223                         return (pos1 < pos2) ? -1 : 1;
4224         }
4225
4226         return 0;
4227 }
4228
4229 static void
4230 copytup_index(Tuplesortstate *state, SortTuple *stup, void *tup)
4231 {
4232         IndexTuple      tuple = (IndexTuple) tup;
4233         unsigned int tuplen = IndexTupleSize(tuple);
4234         IndexTuple      newtuple;
4235         Datum           original;
4236
4237         /* copy the tuple into sort storage */
4238         newtuple = (IndexTuple) MemoryContextAlloc(state->tuplecontext, tuplen);
4239         memcpy(newtuple, tuple, tuplen);
4240         USEMEM(state, GetMemoryChunkSpace(newtuple));
4241         stup->tuple = (void *) newtuple;
4242         /* set up first-column key value */
4243         original = index_getattr(newtuple,
4244                                                          1,
4245                                                          RelationGetDescr(state->indexRel),
4246                                                          &stup->isnull1);
4247
4248         if (!state->sortKeys->abbrev_converter || stup->isnull1)
4249         {
4250                 /*
4251                  * Store ordinary Datum representation, or NULL value.  If there is a
4252                  * converter it won't expect NULL values, and cost model is not
4253                  * required to account for NULL, so in that case we avoid calling
4254                  * converter and just set datum1 to zeroed representation (to be
4255                  * consistent, and to support cheap inequality tests for NULL
4256                  * abbreviated keys).
4257                  */
4258                 stup->datum1 = original;
4259         }
4260         else if (!consider_abort_common(state))
4261         {
4262                 /* Store abbreviated key representation */
4263                 stup->datum1 = state->sortKeys->abbrev_converter(original,
4264                                                                                                                  state->sortKeys);
4265         }
4266         else
4267         {
4268                 /* Abort abbreviation */
4269                 int                     i;
4270
4271                 stup->datum1 = original;
4272
4273                 /*
4274                  * Set state to be consistent with never trying abbreviation.
4275                  *
4276                  * Alter datum1 representation in already-copied tuples, so as to
4277                  * ensure a consistent representation (current tuple was just
4278                  * handled).  It does not matter if some dumped tuples are already
4279                  * sorted on tape, since serialized tuples lack abbreviated keys
4280                  * (TSS_BUILDRUNS state prevents control reaching here in any case).
4281                  */
4282                 for (i = 0; i < state->memtupcount; i++)
4283                 {
4284                         SortTuple  *mtup = &state->memtuples[i];
4285
4286                         tuple = (IndexTuple) mtup->tuple;
4287                         mtup->datum1 = index_getattr(tuple,
4288                                                                                  1,
4289                                                                                  RelationGetDescr(state->indexRel),
4290                                                                                  &mtup->isnull1);
4291                 }
4292         }
4293 }
4294
4295 static void
4296 writetup_index(Tuplesortstate *state, int tapenum, SortTuple *stup)
4297 {
4298         IndexTuple      tuple = (IndexTuple) stup->tuple;
4299         unsigned int tuplen;
4300
4301         tuplen = IndexTupleSize(tuple) + sizeof(tuplen);
4302         LogicalTapeWrite(state->tapeset, tapenum,
4303                                          (void *) &tuplen, sizeof(tuplen));
4304         LogicalTapeWrite(state->tapeset, tapenum,
4305                                          (void *) tuple, IndexTupleSize(tuple));
4306         if (state->randomAccess)        /* need trailing length word? */
4307                 LogicalTapeWrite(state->tapeset, tapenum,
4308                                                  (void *) &tuplen, sizeof(tuplen));
4309
4310         if (!state->slabAllocatorUsed)
4311         {
4312                 FREEMEM(state, GetMemoryChunkSpace(tuple));
4313                 pfree(tuple);
4314         }
4315 }
4316
4317 static void
4318 readtup_index(Tuplesortstate *state, SortTuple *stup,
4319                           int tapenum, unsigned int len)
4320 {
4321         unsigned int tuplen = len - sizeof(unsigned int);
4322         IndexTuple      tuple = (IndexTuple) readtup_alloc(state, tuplen);
4323
4324         LogicalTapeReadExact(state->tapeset, tapenum,
4325                                                  tuple, tuplen);
4326         if (state->randomAccess)        /* need trailing length word? */
4327                 LogicalTapeReadExact(state->tapeset, tapenum,
4328                                                          &tuplen, sizeof(tuplen));
4329         stup->tuple = (void *) tuple;
4330         /* set up first-column key value */
4331         stup->datum1 = index_getattr(tuple,
4332                                                                  1,
4333                                                                  RelationGetDescr(state->indexRel),
4334                                                                  &stup->isnull1);
4335 }
4336
4337 /*
4338  * Routines specialized for DatumTuple case
4339  */
4340
4341 static int
4342 comparetup_datum(const SortTuple *a, const SortTuple *b, Tuplesortstate *state)
4343 {
4344         int                     compare;
4345
4346         compare = ApplySortComparator(a->datum1, a->isnull1,
4347                                                                   b->datum1, b->isnull1,
4348                                                                   state->sortKeys);
4349         if (compare != 0)
4350                 return compare;
4351
4352         /* if we have abbreviations, then "tuple" has the original value */
4353
4354         if (state->sortKeys->abbrev_converter)
4355                 compare = ApplySortAbbrevFullComparator(PointerGetDatum(a->tuple), a->isnull1,
4356                                                                                                 PointerGetDatum(b->tuple), b->isnull1,
4357                                                                                                 state->sortKeys);
4358
4359         return compare;
4360 }
4361
4362 static void
4363 copytup_datum(Tuplesortstate *state, SortTuple *stup, void *tup)
4364 {
4365         /* Not currently needed */
4366         elog(ERROR, "copytup_datum() should not be called");
4367 }
4368
4369 static void
4370 writetup_datum(Tuplesortstate *state, int tapenum, SortTuple *stup)
4371 {
4372         void       *waddr;
4373         unsigned int tuplen;
4374         unsigned int writtenlen;
4375
4376         if (stup->isnull1)
4377         {
4378                 waddr = NULL;
4379                 tuplen = 0;
4380         }
4381         else if (!state->tuples)
4382         {
4383                 waddr = &stup->datum1;
4384                 tuplen = sizeof(Datum);
4385         }
4386         else
4387         {
4388                 waddr = stup->tuple;
4389                 tuplen = datumGetSize(PointerGetDatum(stup->tuple), false, state->datumTypeLen);
4390                 Assert(tuplen != 0);
4391         }
4392
4393         writtenlen = tuplen + sizeof(unsigned int);
4394
4395         LogicalTapeWrite(state->tapeset, tapenum,
4396                                          (void *) &writtenlen, sizeof(writtenlen));
4397         LogicalTapeWrite(state->tapeset, tapenum,
4398                                          waddr, tuplen);
4399         if (state->randomAccess)        /* need trailing length word? */
4400                 LogicalTapeWrite(state->tapeset, tapenum,
4401                                                  (void *) &writtenlen, sizeof(writtenlen));
4402
4403         if (!state->slabAllocatorUsed && stup->tuple)
4404         {
4405                 FREEMEM(state, GetMemoryChunkSpace(stup->tuple));
4406                 pfree(stup->tuple);
4407         }
4408 }
4409
4410 static void
4411 readtup_datum(Tuplesortstate *state, SortTuple *stup,
4412                           int tapenum, unsigned int len)
4413 {
4414         unsigned int tuplen = len - sizeof(unsigned int);
4415
4416         if (tuplen == 0)
4417         {
4418                 /* it's NULL */
4419                 stup->datum1 = (Datum) 0;
4420                 stup->isnull1 = true;
4421                 stup->tuple = NULL;
4422         }
4423         else if (!state->tuples)
4424         {
4425                 Assert(tuplen == sizeof(Datum));
4426                 LogicalTapeReadExact(state->tapeset, tapenum,
4427                                                          &stup->datum1, tuplen);
4428                 stup->isnull1 = false;
4429                 stup->tuple = NULL;
4430         }
4431         else
4432         {
4433                 void       *raddr = readtup_alloc(state, tuplen);
4434
4435                 LogicalTapeReadExact(state->tapeset, tapenum,
4436                                                          raddr, tuplen);
4437                 stup->datum1 = PointerGetDatum(raddr);
4438                 stup->isnull1 = false;
4439                 stup->tuple = raddr;
4440         }
4441
4442         if (state->randomAccess)        /* need trailing length word? */
4443                 LogicalTapeReadExact(state->tapeset, tapenum,
4444                                                          &tuplen, sizeof(tuplen));
4445 }
4446
4447 /*
4448  * Convenience routine to free a tuple previously loaded into sort memory
4449  */
4450 static void
4451 free_sort_tuple(Tuplesortstate *state, SortTuple *stup)
4452 {
4453         FREEMEM(state, GetMemoryChunkSpace(stup->tuple));
4454         pfree(stup->tuple);
4455 }