First, a quick demonstration of what this PoC can do on 1 million
random not-NULL bigints:

set wip_radix_sort = 'off'; select * from test order by a offset 1_000_000_000;
240ms

set wip_radix_sort = 'on'; select * from test order by a offset 1_000_000_000;
140ms

Background: Peter Geoghegan recently mentioned to me off-list an
interesting set of techniques for sorting in the context of databases.
I'm not yet sure how to approach certain aspects of that architecture,
so I won't go into the full picture at this point. However, there is
one piece that already fits well within our existing architecture, and
that is using radix sort on datum1. The basic sequence is:

1. Partition tuples on first key NULL and not-NULL, according to NULLS
FIRST or NULLS LAST.
2. Do normal qsort on the NULL partition using the tiebreak comparator.
3. Create a "conditioned" or "normalized" datum that encodes datum1
such that unsigned comparison is order-preserving, accounting for ASC
/ DESC as well. I've reused space now unused during in-memory not-NULL
sorts:

typedef struct
{
  void     *tuple;      /* the tuple itself */
  Datum    datum1;      /* value of first key column */

  union
  {
    struct
    {
      bool    isnull1;    /* is first key column NULL? */
      int     srctape;    /* source tape number */
    };
    Datum    cond_datum1; /* sort key for radix sort */
  };
} SortTuple;


4. Radix sort on cond_datum1. For the PoC I've based it on the
implementation in "ska sort" [1] (C++, Boost license). For
medium-sized sorts it uses "American flag sort" (there is a paper [3]
co-authored by M. D. McIlroy, same as in the paper we reference for
quicksort). For larger sorts it's similar, but performs multiple
passes, which takes better advantage of modern CPUs. Upon recursion,
sorts on small partitions divert to quicksort. Any necessary tiebreaks
are handled by quicksort, either after the end of radix sort, or when
diverting to small quicksort.
5. Reset isnull1 to "false" before returning to the caller. This also
must be done when diverting to quicksort.

Next steps: Try to find regressions (help welcome here). The v1 patch
has some optimizations, but in other ways things are simple and/or
wasteful. Exactly how things fit together will be informed by what, if
anything, has to be done to avoid regressions. I suspect the challenge
will be multikey sorts when the first key has low cardinality. This is
because tiebreaks are necessarily postponed rather than taken care of
up front. I'm optimistic, since low cardinality cases can be even
faster than our B&M qsort, so we have some headroom:

drop table if exists test;
create unlogged table test (a bigint);
insert into test select
(1_000_000_000 * random())::bigint % 8 -- mod
-- (1_000_000_000 * random())::bigint  -- random, for the case at the top
from generate_series(1,1_000_000,1) i;
vacuum freeze test;

select pg_prewarm('test');
set work_mem = '64MB';

set wip_radix_sort = 'off'; select * from test order by a offset 1_000_000_000;
95ms

set wip_radix_sort = 'on'; select * from test order by a offset 1_000_000_000;
84ms


[1] https://github.com/skarupke/ska_sort/tree/master
[2] https://probablydance.com/2016/12/27/i-wrote-a-faster-sorting-algorithm/
[3] http://static.usenix.org/publications/compsystems/1993/win_mcilroy.pdf

--
John Naylor
Amazon Web Services
From 8185f5b8834d58bddc1c37968d334bd2c97bcc15 Mon Sep 17 00:00:00 2001
From: John Naylor <[email protected]>
Date: Fri, 17 Oct 2025 09:57:43 +0700
Subject: [PATCH v1] Use radix sort when datum1 is an integer type

XXX regression tests don't pass for underspecified queries; this
is expected
---
 src/backend/utils/misc/guc_parameters.dat |   7 +
 src/backend/utils/sort/tuplesort.c        | 641 +++++++++++++++++++++-
 src/include/utils/guc.h                   |   1 +
 src/include/utils/tuplesort.h             |  12 +-
 4 files changed, 641 insertions(+), 20 deletions(-)

diff --git a/src/backend/utils/misc/guc_parameters.dat b/src/backend/utils/misc/guc_parameters.dat
index d6fc8333850..f8fc6c88082 100644
--- a/src/backend/utils/misc/guc_parameters.dat
+++ b/src/backend/utils/misc/guc_parameters.dat
@@ -681,6 +681,13 @@
   boot_val => 'false',
 },
 
+{ name => 'wip_radix_sort', type => 'bool', context => 'PGC_USERSET', group => 'DEVELOPER_OPTIONS',
+  short_desc => 'Test radix sort for debugging.',
+  flags => 'GUC_NOT_IN_SAMPLE',
+  variable => 'wip_radix_sort',
+  boot_val => 'true',
+},
+
 # this is undocumented because not exposed in a standard build
 { name => 'trace_syncscan', type => 'bool', context => 'PGC_USERSET', group => 'DEVELOPER_OPTIONS',
   short_desc => 'Generate debugging output for synchronized scanning.',
diff --git a/src/backend/utils/sort/tuplesort.c b/src/backend/utils/sort/tuplesort.c
index 5d4411dc33f..3b820d245e9 100644
--- a/src/backend/utils/sort/tuplesort.c
+++ b/src/backend/utils/sort/tuplesort.c
@@ -104,6 +104,7 @@
 #include "commands/tablespace.h"
 #include "miscadmin.h"
 #include "pg_trace.h"
+#include "port/pg_bitutils.h"
 #include "storage/shmem.h"
 #include "utils/guc.h"
 #include "utils/memutils.h"
@@ -122,6 +123,7 @@
 
 /* GUC variables */
 bool		trace_sort = false;
+bool		wip_radix_sort = true;
 
 #ifdef DEBUG_BOUNDED_SORT
 bool		optimize_bounded_sort = true;
@@ -615,6 +617,408 @@ qsort_tuple_int32_compare(SortTuple *a, SortTuple *b, Tuplesortstate *state)
 #define ST_DEFINE
 #include "lib/sort_template.h"
 
+
+/*
+ * WIP: For now prefer test coverage of radix sort in Assert builds.
+ * The thresholds are just guesses
+ */
+#ifdef USE_ASSERT_CHECKING
+#define QSORT_THRESHOLD 0
+#define AMERICAN_FLAG_THRESHOLD 0
+#else
+#define QSORT_THRESHOLD 64
+#define AMERICAN_FLAG_THRESHOLD 1500
+#endif
+
+typedef struct PartitionInfo
+{
+	union
+	{
+		size_t		count;
+		size_t		offset;
+	};
+	size_t		next_offset;
+}			PartitionInfo;
+
+static inline uint8_t
+extract_key(Datum key, int level)
+{
+	return (key >> (((SIZEOF_DATUM - 1) - level) * 8)) & 0xFF;
+}
+
+static inline void
+swap(SortTuple *a, SortTuple *b)
+{
+	SortTuple	tmp = *a;
+
+	*a = *b;
+	*b = tmp;
+}
+
+/*
+ * Condition datum to work with pure unsigned comparison,
+ * taking ASC/DESC into account as well.
+ */
+static inline Datum
+condition_datum(Datum orig, SortSupport ssup)
+{
+	Datum		cond_datum1;
+
+	if (ssup->comparator == ssup_datum_signed_cmp)
+	{
+		/* it was already cast to unsigned when stored */
+		cond_datum1 = orig ^ (UINT64CONST(1) << 63);
+	}
+	else if (ssup->comparator == ssup_datum_int32_cmp)
+	{
+		/*
+		 * First normalize to uint32. Technically, we don't need to do this,
+		 * but it forces the upper bytes to remain the same regardless of
+		 * sign.
+		 */
+		uint32		u32 = DatumGetUInt32(orig) ^ ((uint32) 1 << 31);
+
+		cond_datum1 = UInt32GetDatum(u32);
+	}
+	else
+	{
+		Assert(ssup->comparator == ssup_datum_unsigned_cmp);
+		cond_datum1 = orig;
+	}
+
+	if (ssup->ssup_reverse)
+		cond_datum1 = ~cond_datum1;
+
+	return cond_datum1;
+}
+
+/*
+ * Based on implementation in https://github.com/skarupke/ska_sort (Boost license)
+ * TODO: match qsort API with number of elements rather than end pointer
+ */
+static void
+american_flag_sort(SortTuple *begin,
+				   SortTuple *end, int level, Tuplesortstate *state)
+{
+	PartitionInfo partitions[256] = {0};
+	uint8_t		remaining_partitions[256] = {0};
+	size_t		total = 0;
+	int			num_partitions = 0;
+
+	/* count key chunks */
+	for (SortTuple *tup = begin; tup < end; tup++)
+	{
+		uint8		key_chunk;
+
+		key_chunk = extract_key(tup->cond_datum1, level);
+		partitions[key_chunk].count++;
+	}
+
+	/* compute partition offsets */
+	for (int i = 0; i < 256; ++i)
+	{
+		size_t		count = partitions[i].count;
+
+		if (!count)
+			continue;
+		partitions[i].offset = total;
+		total += count;
+		partitions[i].next_offset = total;
+		remaining_partitions[num_partitions] = i;
+		++num_partitions;
+	}
+
+	/* permute tuples to correct partition */
+	if (num_partitions > 1)
+	{
+		uint8_t    *current_block_ptr = remaining_partitions;
+		PartitionInfo *current_block = partitions + *current_block_ptr;
+		uint8_t    *last_block = remaining_partitions + num_partitions - 1;
+		SortTuple  *it = begin;
+		SortTuple  *block_end = begin + current_block->next_offset;
+		SortTuple  *last_element = end - 1;
+
+		for (;;)
+		{
+			PartitionInfo *block = partitions + extract_key(it->cond_datum1, level);
+
+			if (block == current_block)
+			{
+				++it;
+				if (it == last_element)
+					break;
+				else if (it == block_end)
+				{
+					for (;;)
+					{
+						++current_block_ptr;
+						if (current_block_ptr == last_block)
+							goto recurse;
+						current_block = partitions + *current_block_ptr;
+						if (current_block->offset != current_block->next_offset)
+							break;
+					}
+
+					it = begin + current_block->offset;
+					block_end = begin + current_block->next_offset;
+				}
+			}
+			else
+			{
+				size_t		offset = block->offset++;
+
+				swap(it, begin + offset);
+			}
+		}
+	}
+
+recurse:
+	size_t		start_offset = 0;
+	SortTuple  *partition_begin = begin;
+
+	for (uint8_t *it = remaining_partitions, *end = remaining_partitions + num_partitions;
+		 it != end;
+		 ++it)
+	{
+		size_t		end_offset = partitions[*it].next_offset;
+		SortTuple  *partition_end = begin + end_offset;
+		ptrdiff_t	num_elements = end_offset - start_offset;
+
+		if (num_elements > 1)
+		{
+			if (level < SIZEOF_DATUM - 1)
+			{
+				if (num_elements < QSORT_THRESHOLD)
+				{
+					/* restore NOT NULL for fallback qsort */
+					/*
+					 * WIP: Maybe we can have a qsort that skips the NULL
+					 * comparisons, compares directly on cond_datum1, and only
+					 * restores isnull1 if we actually call the tiebreak.
+					 */
+					for (SortTuple *tup = partition_begin;
+						 tup < partition_begin + num_elements;
+						 tup++)
+						tup->isnull1 = false;
+
+					qsort_tuple(partition_begin,
+								num_elements,
+								state->base.comparetup,
+								state);
+				}
+				else
+				{
+					american_flag_sort(partition_begin,
+									   partition_end,
+									   level + 1,
+									   state);
+				}
+			}
+			else if (state->base.onlyKey == NULL)
+			{
+				/*
+				 * Finished radix sort on all bytes of cond_datum1 (possibily
+				 * abbreviated), now qsort with tiebreak comparator.
+				 */
+
+				/* first restore NOT NULL for later comparators */
+				for (SortTuple *tup = partition_begin;
+					 tup < partition_begin + num_elements;
+					 tup++)
+					tup->isnull1 = false;
+
+				qsort_tuple(partition_begin,
+							num_elements,
+							state->base.comparetup_tiebreak,
+							state);
+			}
+		}
+		start_offset = end_offset;
+		partition_begin = partition_end;
+	}
+}
+
+/*
+ * Based on implementation in https://github.com/skarupke/ska_sort (Boost license),
+ * with the following changes:
+ *  - unroll loop in counting step
+ *  - count sorted partitions in every pass, rather than maintaining list of unsorted partitions
+ * TODO: match qsort API with number of elements rather than end pointer
+ */
+static void
+ska_byte_sort(SortTuple *begin,
+			  SortTuple *end, int level, Tuplesortstate *state)
+{
+	/* size_t		counts0[256] = {0}; */
+	size_t		counts1[256] = {0};
+	size_t		counts2[256] = {0};
+	size_t		counts3[256] = {0};
+	PartitionInfo partitions[256] = {0};
+	uint8_t		remaining_partitions[256] = {0};
+	size_t		total = 0;
+	int			num_partitions = 0;
+	int			num_remaining;
+	SortTuple  *ctup;
+
+	/* count key chunks, unrolled for speed */
+
+	for (ctup = begin; ctup + 4 < end; ctup += 4)
+	{
+		uint8		key_chunk0 = extract_key((ctup + 0)->cond_datum1, level);
+		uint8		key_chunk1 = extract_key((ctup + 1)->cond_datum1, level);
+		uint8		key_chunk2 = extract_key((ctup + 2)->cond_datum1, level);
+		uint8		key_chunk3 = extract_key((ctup + 3)->cond_datum1, level);
+
+		partitions[key_chunk0].count++;
+		counts1[key_chunk1]++;
+		counts2[key_chunk2]++;
+		counts3[key_chunk3]++;
+
+	}
+
+	for (size_t i = 0; i < 256; i++)
+		partitions[i].count += counts1[i] + counts2[i] + counts3[i];
+
+	for (; ctup < end; ctup++)
+	{
+		uint8		key_chunk;
+
+		key_chunk = extract_key(ctup->cond_datum1, level);
+		partitions[key_chunk].count++;
+	}
+
+	/* compute partition offsets */
+	for (int i = 0; i < 256; ++i)
+	{
+		size_t		count = partitions[i].count;
+
+		if (count)
+		{
+			partitions[i].offset = total;
+			total += count;
+			remaining_partitions[num_partitions] = i;
+			++num_partitions;
+		}
+		partitions[i].next_offset = total;
+	}
+
+	num_remaining = num_partitions;
+
+	/*
+	 * Permute tuples to correct partition. If we started with one partition,
+	 * there is nothing to do. If a permutation from a previous iteration
+	 * results in a single partition that hasn't been marked as sorted, we
+	 * know it's actually sorted.
+	 */
+	while (num_remaining > 1)
+	{
+		/*
+		 * We can only exit the loop when all partitions are sorted, so must
+		 * reset every iteration
+		 */
+		num_remaining = num_partitions;
+
+		for (int i = 0; i < num_partitions; i++)
+		{
+			uint8		idx = remaining_partitions[i];
+
+			PartitionInfo part = partitions[idx];
+
+			for (SortTuple *st = begin + part.offset;
+				 st < begin + part.next_offset;
+				 st++)
+			{
+				uint8		this_partition = extract_key(st->cond_datum1, level);
+				size_t		offset = partitions[this_partition].offset++;
+
+				Assert(begin + offset < end);
+				swap(st, begin + offset);
+			};
+
+			if (part.offset == part.next_offset)
+			{
+				/* partition is sorted; skip */
+				num_remaining--;
+			}
+		}
+	}
+
+	{
+		size_t		start_offset = 0;
+		SortTuple  *partition_begin = begin;
+
+		for (uint8_t *it = remaining_partitions, *end = remaining_partitions + num_partitions;
+			 it != end;
+			 ++it)
+		{
+			size_t		end_offset = partitions[*it].next_offset;
+			SortTuple  *partition_end = begin + end_offset;
+			ptrdiff_t	num_elements = end_offset - start_offset;
+
+			if (num_elements > 1)
+			{
+				if (level < SIZEOF_DATUM - 1)
+				{
+					if (num_elements < QSORT_THRESHOLD)
+					{
+						/* restore NOT NULL for fallback qsort */
+						/*
+						 * WIP: Maybe we can have a qsort that skips the NULL
+						 * comparisons, compares directly on cond_datum1, and
+						 * only restores isnull1 if we actually call the
+						 * tiebreak.
+						 */
+						for (SortTuple *tup = partition_begin;
+							 tup < partition_begin + num_elements;
+							 tup++)
+							tup->isnull1 = false;
+
+						qsort_tuple(partition_begin,
+									num_elements,
+									state->base.comparetup,
+									state);
+					}
+					else if (num_elements < AMERICAN_FLAG_THRESHOLD)
+					{
+						american_flag_sort(partition_begin,
+										   partition_end,
+										   level + 1,
+										   state);
+					}
+					else
+					{
+						ska_byte_sort(partition_begin,
+									  partition_end,
+									  level + 1,
+									  state);
+					}
+				}
+				else if (state->base.onlyKey == NULL)
+				{
+					/*
+					 * Finished radix sort on all bytes of cond_datum1
+					 * (possibily abbreviated), now qsort with tiebreak
+					 * comparator.
+					 */
+
+					/* first restore NOT NULL for later comparators */
+					for (SortTuple *tup = partition_begin;
+						 tup < partition_begin + num_elements;
+						 tup++)
+						tup->isnull1 = false;
+
+					qsort_tuple(partition_begin,
+								num_elements,
+								state->base.comparetup_tiebreak,
+								state);
+				}
+			}
+			start_offset = end_offset;
+			partition_begin = partition_end;
+		}
+	}
+}
+
 /*
  *		tuplesort_begin_xxx
  *
@@ -2663,8 +3067,192 @@ sort_bounded_heap(Tuplesortstate *state)
 	state->boundUsed = true;
 }
 
+/* WIP: allow turning common prefix skipping off for testing */
+#define COMMON_PREFIX
+
+/*
+ * Prepare SortTuples for radix sort before dispatch to the actual sort.
+ */
+static void
+radixsort_tuple(Tuplesortstate *state)
+{
+	SortSupportData ssup = state->base.sortKeys[0];
+
+	bool		nulls_first = ssup.ssup_nulls_first;
+	SortTuple  *first = state->memtuples;
+	SortTuple  *last = state->memtuples + state->memtupcount;
+	SortTuple  *not_null_start;
+	size_t		d1,
+				d2,
+				not_null_count;
+#ifdef COMMON_PREFIX
+	Datum		first_datum = 0;
+	Datum		common_upper_bits = 0;
+#endif
+	int			common_prefix;
+
+	/*
+	 * Partition by isnull1, since we can only radix sort on non-NULL
+	 * elements.
+	 */
+
+	/*
+	 * Find the leftmost NOT NULL tuple if NULLS FIRST, or leftmost NULL
+	 * element if NULLS LAST.
+	 */
+	while (first < last && first->isnull1 == nulls_first)
+		first++;
+
+	/*
+	 * XXX We must start "last" after the final tuple to maintain the
+	 * invariant that it ends up one after the first partition, and the first
+	 * partition may correspond to the entire array. If "first" isn't gotten
+	 * this far, we need to pre-decrement "last" before beginning its loop.
+	 */
+	if (first < last)
+		last--;
+
+	/*
+	 * Find the rightmost NULL tuple if NULLS FIRST, or rightmost NOT NULL
+	 * tuple if NULLS LAST.
+	 */
+	while (first < last && last->isnull1 != nulls_first)
+		last--;
+
+	/* swap pairs of tuples that are in the wrong order */
+	while (first < last)
+	{
+		swap(first, last);
+		while (first < last && first->isnull1 == nulls_first)
+			first++;
+		while (first < last && last->isnull1 != nulls_first)
+			last--;
+	}
+
+	d1 = last - state->memtuples;
+	d2 = state->memtupcount - d1;
+
+	Assert(last = first);
+	Assert(last + d2 == state->memtuples + state->memtupcount);
+	for (SortTuple *pm = state->memtuples;
+		 pm < state->memtuples + d1;
+		 pm++)
+		Assert(pm->isnull1 == nulls_first);
+	for (SortTuple *pm = last;
+		 pm < last + d2;
+		 pm++)
+		Assert(pm->isnull1 != nulls_first);
+
+	/*
+	 * Sort null partition using tiebreak comparator. XXX this will repeat the
+	 * NULL check for abbreviated keys.
+	 */
+	if (nulls_first)
+	{
+		qsort_tuple(state->memtuples,
+					d1,
+					state->base.comparetup_tiebreak,
+					state);
+		not_null_start = last;
+		not_null_count = d2;
+	}
+	else
+	{
+		qsort_tuple(last,
+					d2,
+					state->base.comparetup_tiebreak,
+					state);
+		not_null_start = state->memtuples;
+		not_null_count = d1;
+	}
+
+	/*
+	 * Condition datum so that unsigned comparision is order-preserving, and
+	 * compute the common prefix to skip unproductive recursion steps.
+	 */
+	for (SortTuple *tup = not_null_start;
+		 tup < not_null_start + not_null_count;
+		 tup++)
+	{
+		Datum		cond_datum1 = condition_datum(tup->datum1, &ssup);
+#ifdef COMMON_PREFIX
+		if (tup == not_null_start)
+		{
+			/* Need to start with some value, may as well be the first one. */
+			first_datum = cond_datum1;
+		}
+		else
+		{
+			Datum		this_common_bits;
+
+			/* The bits in common will be zero */
+			this_common_bits = first_datum ^ cond_datum1;
+
+			/*
+			 * We're really only interested in the case where the rightmost
+			 * one bit is further right, but this branch should be rare enough
+			 * not to waste cycles trying harder.
+			 */
+			if (this_common_bits > common_upper_bits)
+				common_upper_bits = this_common_bits;
+		}
+#endif
+		tup->cond_datum1 = cond_datum1;
+	}
+
+	/*
+	 * The upper bits are zero where all values are the same, if any. Turn the
+	 * byte position of the rightmost one bit into the byte where radix sort
+	 * should start bucketing. OR-ing in the lowest bit guards against
+	 * undefined behavior without changing the result.
+	 */
+#ifdef COMMON_PREFIX
+	common_prefix = sizeof(Datum) - 1 -
+		(pg_leftmost_one_pos64(common_upper_bits | 1) / BITS_PER_BYTE);
+#else
+	common_prefix = 0;
+#endif
+	/* perform the radix sort on the not-NULL partition */
+	ska_byte_sort(not_null_start,
+				  not_null_start + not_null_count,
+				  common_prefix,
+				  state);
+
+	/*
+	 * Restore fields that were overwritten with temporary conditioned datum1
+	 */
+	for (SortTuple *tup = not_null_start;
+		 tup < not_null_start + not_null_count;
+		 tup++)
+	{
+		/* need to restore NOT NULL */
+		tup->isnull1 = false;
+		/* be tidy */
+		tup->srctape = 0;
+	}
+}
+
+/* Verify sort using standard comparator. */
+static void
+check_sorted(Tuplesortstate *state)
+{
+#ifdef USE_ASSERT_CHECKING
+	for (SortTuple *pm = state->memtuples + 1;
+		 pm < state->memtuples + state->memtupcount;
+		 pm++)
+	{
+#if 0
+		Assert(COMPARETUP(state, pm - 1, pm) <= 0);
+#else
+		if (COMPARETUP(state, pm - 1, pm) > 0)
+			elog(ERROR, "SORT FAILED");
+#endif
+	}
+#endif
+}
+
 /*
- * Sort all memtuples using specialized qsort() routines.
+ * Sort all memtuples using specialized routines.
  *
  * Quicksort is used for small in-memory sorts, and external sort runs.
  */
@@ -2681,26 +3269,43 @@ tuplesort_sort_memtuples(Tuplesortstate *state)
 		 */
 		if (state->base.haveDatum1 && state->base.sortKeys)
 		{
-			if (state->base.sortKeys[0].comparator == ssup_datum_unsigned_cmp)
-			{
-				qsort_tuple_unsigned(state->memtuples,
-									 state->memtupcount,
-									 state);
-				return;
-			}
-			else if (state->base.sortKeys[0].comparator == ssup_datum_signed_cmp)
+			SortSupportData ssup = state->base.sortKeys[0];
+
+			if (wip_radix_sort)
 			{
-				qsort_tuple_signed(state->memtuples,
-								   state->memtupcount,
-								   state);
-				return;
+				if (state->memtupcount > QSORT_THRESHOLD &&
+					(ssup.comparator == ssup_datum_unsigned_cmp ||
+					 ssup.comparator == ssup_datum_signed_cmp ||
+					 ssup.comparator == ssup_datum_int32_cmp))
+				{
+					radixsort_tuple(state);
+					check_sorted(state);
+					return;
+				}
 			}
-			else if (state->base.sortKeys[0].comparator == ssup_datum_int32_cmp)
+			else
 			{
-				qsort_tuple_int32(state->memtuples,
-								  state->memtupcount,
-								  state);
-				return;
+				if (state->base.sortKeys[0].comparator == ssup_datum_unsigned_cmp)
+				{
+					qsort_tuple_unsigned(state->memtuples,
+										 state->memtupcount,
+										 state);
+					return;
+				}
+				else if (state->base.sortKeys[0].comparator == ssup_datum_signed_cmp)
+				{
+					qsort_tuple_signed(state->memtuples,
+									   state->memtupcount,
+									   state);
+					return;
+				}
+				else if (state->base.sortKeys[0].comparator == ssup_datum_int32_cmp)
+				{
+					qsort_tuple_int32(state->memtuples,
+									  state->memtupcount,
+									  state);
+					return;
+				}
 			}
 		}
 
diff --git a/src/include/utils/guc.h b/src/include/utils/guc.h
index f21ec37da89..bc6f7fa60f3 100644
--- a/src/include/utils/guc.h
+++ b/src/include/utils/guc.h
@@ -324,6 +324,7 @@ extern PGDLLIMPORT int tcp_user_timeout;
 extern PGDLLIMPORT char *role_string;
 extern PGDLLIMPORT bool in_hot_standby_guc;
 extern PGDLLIMPORT bool trace_sort;
+extern PGDLLIMPORT bool wip_radix_sort;
 
 #ifdef DEBUG_BOUNDED_SORT
 extern PGDLLIMPORT bool optimize_bounded_sort;
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index ef79f259f93..b2ecbbc9e51 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -149,8 +149,16 @@ typedef struct
 {
 	void	   *tuple;			/* the tuple itself */
 	Datum		datum1;			/* value of first key column */
-	bool		isnull1;		/* is first key column NULL? */
-	int			srctape;		/* source tape number */
+
+	union
+	{
+		struct
+		{
+			bool		isnull1;		/* is first key column NULL? */
+			int			srctape;		/* source tape number */
+		};
+		Datum		cond_datum1;		/* sort key for radix sort */
+	};
 } SortTuple;
 
 typedef int (*SortTupleComparator) (const SortTuple *a, const SortTuple *b,
-- 
2.51.0

Reply via email to