From 661bbda2e8cf5e3cb2e1f8214e3a7d0445c67171 Mon Sep 17 00:00:00 2001 From: wuyuechuan Date: Tue, 24 Nov 2020 11:07:49 +0800 Subject: [PATCH 1/2] add shared tuplestore --- src/common/backend/utils/sort/Makefile | 2 +- .../backend/utils/sort/sharedtuplestore.cpp | 583 ++++++++++++++++++ src/gausskernel/storage/lmgr/lwlock.cpp | 1 + src/include/storage/lwlock.h | 2 +- src/include/utils/sharedtuplestore.h | 51 ++ src/tools/pgindent/typedefs.list | 2 + 6 files changed, 639 insertions(+), 2 deletions(-) create mode 100644 src/common/backend/utils/sort/sharedtuplestore.cpp create mode 100644 src/include/utils/sharedtuplestore.h diff --git a/src/common/backend/utils/sort/Makefile b/src/common/backend/utils/sort/Makefile index 7dccb789f..d25a5c589 100644 --- a/src/common/backend/utils/sort/Makefile +++ b/src/common/backend/utils/sort/Makefile @@ -21,7 +21,7 @@ ifneq "$(MAKECMDGOALS)" "clean" endif endif endif -OBJS = logtape.o sortsupport.o tuplesort.o tuplestore.o batchsort.o batchstore.o rowstore.o +OBJS = logtape.o sharedtuplestore.o sortsupport.o tuplesort.o tuplestore.o batchsort.o batchstore.o rowstore.o tuplesort.o: qsort_tuple.cpp diff --git a/src/common/backend/utils/sort/sharedtuplestore.cpp b/src/common/backend/utils/sort/sharedtuplestore.cpp new file mode 100644 index 000000000..4ec3ea8d4 --- /dev/null +++ b/src/common/backend/utils/sort/sharedtuplestore.cpp @@ -0,0 +1,583 @@ +/*------------------------------------------------------------------------- + * + * sharedtuplestore.c + * Simple mechanism for sharing tuples between backends. + * + * This module contains a shared temporary tuple storage mechanism providing + * a parallel-aware subset of the features of tuplestore.c. Multiple backends + * can write to a SharedTuplestore, and then multiple backends can later scan + * the stored tuples. Currently, the only scan type supported is a parallel + * scan where each backend reads an arbitrary subset of the tuples that were + * written. + * + * Portions Copyright (c) 1996-2020, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * IDENTIFICATION + * src/backend/utils/sort/sharedtuplestore.c + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "access/htup.h" +#include "miscadmin.h" +#include "storage/buffile.h" +#include "storage/lwlock.h" +#include "storage/sharedfileset.h" +#include "utils/sharedtuplestore.h" + +/* + * The size of chunks, in pages. This is somewhat arbitrarily set to match + * the size of HASH_CHUNK, so that Parallel Hash obtains new chunks of tuples + * at approximately the same rate as it allocates new chunks of memory to + * insert them into. + */ +#define STS_CHUNK_PAGES 4 +#define STS_CHUNK_HEADER_SIZE offsetof(SharedTuplestoreChunk, data) +#define STS_CHUNK_DATA_SIZE (STS_CHUNK_PAGES * BLCKSZ - STS_CHUNK_HEADER_SIZE) +#define WRITE_CHUNK_LEN (STS_CHUNK_PAGES * BLCKSZ) + +/* Chunk written to disk. */ +typedef struct SharedTuplestoreChunk { + int ntuples; /* Number of tuples in this chunk. */ + int overflow; /* If overflow, how many including this one? */ + char data[FLEXIBLE_ARRAY_MEMBER]; +} SharedTuplestoreChunk; + +/* Per-participant shared state. */ +typedef struct SharedTuplestoreParticipant { + LWLock lock; + BlockNumber read_page; /* Page number for next read. */ + BlockNumber npages; /* Number of pages written. */ + bool writing; /* Used only for assertions. */ +} SharedTuplestoreParticipant; + +/* The control object that lives in shared memory. */ +struct SharedTuplestore { + int nparticipants; /* Number of participants that can write. */ + int flags; /* Flag bits from SHARED_TUPLESTORE_XXX */ + size_t meta_data_size; /* Size of per-tuple header. */ + char name[NAMEDATALEN]; /* A name for this tuplestore. */ + + /* Followed by per-participant shared state. */ + SharedTuplestoreParticipant participants[FLEXIBLE_ARRAY_MEMBER]; +}; + +/* Per-participant state that lives in backend-local memory. */ +struct SharedTuplestoreAccessor { + int participant; /* My participant number. */ + SharedTuplestore* sts; /* The shared state. */ + SharedFileSet* fileset; /* The SharedFileSet holding files. */ + MemoryContext context; /* Memory context for buffers. */ + + /* State for reading. */ + int read_participant; /* The current participant to read from. */ + BufFile* read_file; /* The current file to read from. */ + int read_ntuples_available; /* The number of tuples in chunk. */ + int read_ntuples; /* How many tuples have we read from chunk? */ + size_t read_bytes; /* How many bytes have we read from chunk? */ + char* read_buffer; /* A buffer for loading tuples. */ + size_t read_buffer_size; + BlockNumber read_next_page; /* Lowest block we'll consider reading. */ + + /* State for writing. */ + SharedTuplestoreChunk* write_chunk; /* Buffer for writing. */ + BufFile* write_file; /* The current file to write to. */ + char* write_pointer; /* Current write pointer within chunk. */ + char* write_end; /* One past the end of the current chunk. */ +}; + +static void sts_filename(char* name, size_t length, SharedTuplestoreAccessor* accessor, int participant); + +/* + * Return the amount of shared memory required to hold SharedTuplestore for a + * given number of participants. + */ +size_t sts_estimate(int participants) +{ + return (size_t)(offsetof(SharedTuplestore, participants) + sizeof(SharedTuplestoreParticipant) * participants); +} + +/* + * Initialize a SharedTuplestore in existing shared memory. There must be + * space for sts_estimate(participants) bytes. If flags includes the value + * SHARED_TUPLESTORE_SINGLE_PASS, the files may in future be removed more + * eagerly (but this isn't yet implemented). + * + * Tuples that are stored may optionally carry a piece of fixed sized + * meta-data which will be retrieved along with the tuple. This is useful for + * the hash values used in multi-batch hash joins, but could have other + * applications. + * + * The caller must supply a SharedFileSet, which is essentially a directory + * that will be cleaned up automatically, and a name which must be unique + * across all SharedTuplestores created in the same SharedFileSet. + */ +SharedTuplestoreAccessor* sts_initialize(SharedTuplestore* sts, int participants, int my_participant_number, + size_t meta_data_size, int flags, SharedFileSet* fileset, const char* name) +{ + SharedTuplestoreAccessor* accessor = NULL; + int i; + + Assert(my_participant_number < participants); + + sts->nparticipants = participants; + sts->meta_data_size = meta_data_size; + sts->flags = flags; + + if (strlen(name) > sizeof(sts->name) - 1) { + elog(ERROR, "SharedTuplestore name too long"); + } + errno_t rc = strcpy_s(sts->name, NAMEDATALEN, name); + securec_check(rc, "\0", "\0"); + + /* + * Limit meta-data so it + tuple size always fits into a single chunk. + * sts_puttuple() and sts_read_tuple() could be made to support scenarios + * where that's not the case, but it's not currently required. If so, + * meta-data size probably should be made variable, too. + */ + if (meta_data_size + sizeof(uint32) >= STS_CHUNK_DATA_SIZE) { + elog(ERROR, "meta-data too long"); + } + + for (i = 0; i < participants; ++i) { + LWLockInitialize(&sts->participants[i].lock, LWTRANCHE_SHARED_TUPLESTORE); + sts->participants[i].read_page = 0; + sts->participants[i].writing = false; + } + + accessor = (SharedTuplestoreAccessor*)palloc0(sizeof(SharedTuplestoreAccessor)); + accessor->participant = my_participant_number; + accessor->sts = sts; + accessor->fileset = fileset; + accessor->context = CurrentMemoryContext; + + return accessor; +} + +/* + * Attach to a SharedTuplestore that has been initialized by another backend, + * so that this backend can read and write tuples. + */ +SharedTuplestoreAccessor* sts_attach(SharedTuplestore* sts, int my_participant_number, SharedFileSet* fileset) +{ + SharedTuplestoreAccessor* accessor = NULL; + + Assert(my_participant_number < sts->nparticipants); + + accessor = (SharedTuplestoreAccessor*)palloc0(sizeof(SharedTuplestoreAccessor)); + accessor->participant = my_participant_number; + accessor->sts = sts; + accessor->fileset = fileset; + accessor->context = CurrentMemoryContext; + + return accessor; +} + +static void sts_flush_chunk(SharedTuplestoreAccessor* accessor) +{ + size_t size; + + size = WRITE_CHUNK_LEN; + BufFileWrite(accessor->write_file, accessor->write_chunk, size); + int rc = memset_s(accessor->write_chunk, size, 0, size); + securec_check(rc, "", ""); + + accessor->write_pointer = &accessor->write_chunk->data[0]; + accessor->sts->participants[accessor->participant].npages += STS_CHUNK_PAGES; +} + +/* + * Finish writing tuples. This must be called by all backends that have + * written data before any backend begins reading it. + */ +void sts_end_write(SharedTuplestoreAccessor* accessor) +{ + if (accessor->write_file != NULL) { + sts_flush_chunk(accessor); + BufFileClose(accessor->write_file); + pfree(accessor->write_chunk); + accessor->write_chunk = NULL; + accessor->write_file = NULL; + accessor->sts->participants[accessor->participant].writing = false; + } +} + +/* + * Prepare to rescan. Only one participant must call this. After it returns, + * all participants may call sts_begin_parallel_scan() and then loop over + * sts_parallel_scan_next(). This function must not be called concurrently + * with a scan, and synchronization to avoid that is the caller's + * responsibility. + */ +void sts_reinitialize(SharedTuplestoreAccessor* accessor) +{ + int i; + + /* + * Reset the shared read head for all participants' files. Also set the + * initial chunk size to the minimum (any increases from that size will be + * recorded in chunk_expansion_log). + */ + for (i = 0; i < accessor->sts->nparticipants; ++i) { + accessor->sts->participants[i].read_page = 0; + } +} + +/* + * Begin scanning the contents in parallel. + */ +void sts_begin_parallel_scan(SharedTuplestoreAccessor* accessor) +{ + int i PG_USED_FOR_ASSERTS_ONLY; + + /* End any existing scan that was in progress. */ + sts_end_parallel_scan(accessor); + + /* + * Any backend that might have written into this shared tuplestore must + * have called sts_end_write(), so that all buffers are flushed and the + * files have stopped growing. + */ + for (i = 0; i < accessor->sts->nparticipants; ++i) { + Assert(!accessor->sts->participants[i].writing); + } + + /* + * We will start out reading the file that THIS backend wrote. There may + * be some caching locality advantage to that. + */ + accessor->read_participant = accessor->participant; + accessor->read_file = NULL; + accessor->read_next_page = 0; +} + +/* + * Finish a parallel scan, freeing associated backend-local resources. + */ +void sts_end_parallel_scan(SharedTuplestoreAccessor* accessor) +{ + /* + * Here we could delete all files if SHARED_TUPLESTORE_SINGLE_PASS, but + * we'd probably need a reference count of current parallel scanners so we + * could safely do it only when the reference count reaches zero. + */ + if (accessor->read_file != NULL) { + BufFileClose(accessor->read_file); + accessor->read_file = NULL; + } +} + +/* + * Write a tuple. If a meta-data size was provided to sts_initialize, then a + * pointer to meta data of that size must be provided. + */ +void sts_puttuple(SharedTuplestoreAccessor* accessor, void* meta_data, MinimalTuple tuple) +{ + size_t size; + errno_t rc; + + /* Do we have our own file yet? */ + if (accessor->write_file == NULL) { + SharedTuplestoreParticipant* participant = NULL; + char name[MAXPGPATH]; + + /* Create one. Only this backend will write into it. */ + sts_filename(name, MAXPGPATH, accessor, accessor->participant); + accessor->write_file = BufFileCreateShared(accessor->fileset, name); + + /* Set up the shared state for this backend's file. */ + participant = &accessor->sts->participants[accessor->participant]; + participant->writing = true; /* for assertions only */ + } + + size_t total_size = STS_CHUNK_DATA_SIZE; + /* Do we have space? */ + size = accessor->sts->meta_data_size + tuple->t_len; + if (accessor->write_pointer + size >= accessor->write_end) { + if (accessor->write_chunk == NULL) { + /* First time through. Allocate chunk. */ + accessor->write_chunk = (SharedTuplestoreChunk*)MemoryContextAllocZero(accessor->context, WRITE_CHUNK_LEN); + accessor->write_chunk->ntuples = 0; + accessor->write_pointer = &accessor->write_chunk->data[0]; + accessor->write_end = (char*)accessor->write_chunk + WRITE_CHUNK_LEN; + } else { + /* See if flushing helps. */ + sts_flush_chunk(accessor); + } + + /* It may still not be enough in the case of a gigantic tuple. */ + if (accessor->write_pointer + size >= accessor->write_end) { + long written; + + /* + * We'll write the beginning of the oversized tuple, and then + * write the rest in some number of 'overflow' chunks. + * + * sts_initialize() verifies that the size of the tuple + + * meta-data always fits into a chunk. Because the chunk has been + * flushed above, we can be sure to have all of a chunk's usable + * space available. + */ + Assert(accessor->write_pointer + accessor->sts->meta_data_size + sizeof(uint32) < accessor->write_end); + + /* Write the meta-data as one chunk. */ + if (accessor->sts->meta_data_size > 0) { + rc = memcpy_s(accessor->write_pointer, total_size, meta_data, accessor->sts->meta_data_size); + securec_check(rc, "\0", "\0"); + } + + /* + * Write as much of the tuple as we can fit. This includes the + * tuple's size at the start. + */ + written = accessor->write_end - accessor->write_pointer - accessor->sts->meta_data_size; + rc = memcpy_s(accessor->write_pointer + accessor->sts->meta_data_size, + total_size - accessor->sts->meta_data_size, + tuple, + written); + securec_check(rc, "\0", "\0"); + + ++accessor->write_chunk->ntuples; + size -= accessor->sts->meta_data_size; + size -= written; + /* Now write as many overflow chunks as we need for the rest. */ + while (size != 0) { + size_t written_this_chunk; + + sts_flush_chunk(accessor); + + /* + * How many overflow chunks to go? This will allow readers to + * skip all of them at once instead of reading each one. + */ + accessor->write_chunk->overflow = (size + STS_CHUNK_DATA_SIZE - 1) / STS_CHUNK_DATA_SIZE; + written_this_chunk = Min((size_t)(accessor->write_end - accessor->write_pointer), size); + rc = memcpy_s(accessor->write_pointer, total_size, (char*)tuple + written, written_this_chunk); + securec_check(rc, "\0", "\0"); + accessor->write_pointer += written_this_chunk; + size -= written_this_chunk; + written += written_this_chunk; + } + return; + } + } + + /* Copy meta-data and tuple into buffer. */ + if (accessor->sts->meta_data_size > 0) { + rc = memcpy_s(accessor->write_pointer, total_size, meta_data, accessor->sts->meta_data_size); + securec_check(rc, "\0", "\0"); + } + rc = memcpy_s(accessor->write_pointer + accessor->sts->meta_data_size, + total_size - accessor->sts->meta_data_size, + tuple, + tuple->t_len); + securec_check(rc, "\0", "\0"); + accessor->write_pointer += size; + ++accessor->write_chunk->ntuples; +} + +static MinimalTuple sts_read_tuple(SharedTuplestoreAccessor* accessor, void* meta_data) +{ + MinimalTuple tuple = NULL; + uint32 size; + size_t remaining_size; + size_t this_chunk_size; + char* destination = NULL; + + /* + * We'll keep track of bytes read from this chunk so that we can detect an + * overflowing tuple and switch to reading overflow pages. + */ + if (accessor->sts->meta_data_size > 0) { + if (BufFileRead(accessor->read_file, meta_data, accessor->sts->meta_data_size) != + accessor->sts->meta_data_size) { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from shared tuplestore temporary file"), + errdetail_internal("Short read while reading meta-data."))); + } + accessor->read_bytes += accessor->sts->meta_data_size; + } + if (BufFileRead(accessor->read_file, &size, sizeof(size)) != sizeof(size)) { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from shared tuplestore temporary file"), + errdetail_internal("Short read while reading size."))); + } + accessor->read_bytes += sizeof(size); + if (size > accessor->read_buffer_size) { + size_t new_read_buffer_size; + + if (accessor->read_buffer != NULL) + pfree(accessor->read_buffer); + new_read_buffer_size = Max(size, accessor->read_buffer_size * 2); + accessor->read_buffer = (char*)MemoryContextAlloc(accessor->context, new_read_buffer_size); + accessor->read_buffer_size = new_read_buffer_size; + } + remaining_size = size - sizeof(uint32); + this_chunk_size = Min(remaining_size, BLCKSZ * STS_CHUNK_PAGES - accessor->read_bytes); + destination = accessor->read_buffer + sizeof(uint32); + if (BufFileRead(accessor->read_file, destination, this_chunk_size) != this_chunk_size) { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from shared tuplestore temporary file"), + errdetail_internal("Short read while reading tuple."))); + } + accessor->read_bytes += this_chunk_size; + remaining_size -= this_chunk_size; + destination += this_chunk_size; + ++accessor->read_ntuples; + + /* Check if we need to read any overflow chunks. */ + while (remaining_size > 0) { + /* We are now positioned at the start of an overflow chunk. */ + SharedTuplestoreChunk chunk_header; + + if (BufFileRead(accessor->read_file, &chunk_header, STS_CHUNK_HEADER_SIZE) != STS_CHUNK_HEADER_SIZE) { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from shared tuplestore temporary file"), + errdetail_internal("Short read while reading overflow chunk header."))); + } + accessor->read_bytes = STS_CHUNK_HEADER_SIZE; + if (chunk_header.overflow == 0) { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("unexpected chunk in shared tuplestore temporary file"), + errdetail_internal("Expected overflow chunk."))); + } + accessor->read_next_page += STS_CHUNK_PAGES; + this_chunk_size = Min(remaining_size, BLCKSZ * STS_CHUNK_PAGES - STS_CHUNK_HEADER_SIZE); + if (BufFileRead(accessor->read_file, destination, this_chunk_size) != this_chunk_size) { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from shared tuplestore temporary file"), + errdetail_internal("Short read while reading tuple."))); + } + accessor->read_bytes += this_chunk_size; + remaining_size -= this_chunk_size; + destination += this_chunk_size; + + /* + * These will be used to count regular tuples following the oversized + * tuple that spilled into this overflow chunk. + */ + accessor->read_ntuples = 0; + accessor->read_ntuples_available = chunk_header.ntuples; + } + + tuple = (MinimalTuple)accessor->read_buffer; + tuple->t_len = size; + + return tuple; +} + +/* + * Get the next tuple in the current parallel scan. + */ +MinimalTuple sts_parallel_scan_next(SharedTuplestoreAccessor* accessor, void* meta_data) +{ + SharedTuplestoreParticipant* p = NULL; + BlockNumber read_page; + bool eof = false; + + for (;;) { + /* Can we read more tuples from the current chunk? */ + if (accessor->read_ntuples < accessor->read_ntuples_available) { + return sts_read_tuple(accessor, meta_data); + } + + /* Find the location of a new chunk to read. */ + p = &accessor->sts->participants[accessor->read_participant]; + + LWLockAcquire(&p->lock, LW_EXCLUSIVE); + /* We can skip directly past overflow pages we know about. */ + if (p->read_page < accessor->read_next_page) { + p->read_page = accessor->read_next_page; + } + eof = p->read_page >= p->npages; + if (!eof) { + /* Claim the next chunk. */ + read_page = p->read_page; + /* Advance the read head for the next reader. */ + p->read_page += STS_CHUNK_PAGES; + accessor->read_next_page = p->read_page; + } + LWLockRelease(&p->lock); + + if (!eof) { + SharedTuplestoreChunk chunk_header; + size_t nread; + + /* Make sure we have the file open. */ + if (accessor->read_file == NULL) { + char name[MAXPGPATH]; + + sts_filename(name, MAXPGPATH, accessor, accessor->read_participant); + accessor->read_file = BufFileOpenShared(accessor->fileset, name); + } + + /* Seek and load the chunk header. */ + if (BufFileSeekBlock(accessor->read_file, read_page) != 0) { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not seek block %u in shared tuplestore temporary file", read_page))); + } + nread = BufFileRead(accessor->read_file, &chunk_header, STS_CHUNK_HEADER_SIZE); + if (nread != STS_CHUNK_HEADER_SIZE) { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from shared tuplestore temporary file: read only %zu of %zu bytes", + nread, + STS_CHUNK_HEADER_SIZE))); + } + + /* + * If this is an overflow chunk, we skip it and any following + * overflow chunks all at once. + */ + if (chunk_header.overflow > 0) { + accessor->read_next_page = (unsigned int)(read_page + chunk_header.overflow * STS_CHUNK_PAGES); + continue; + } + + accessor->read_ntuples = 0; + accessor->read_ntuples_available = chunk_header.ntuples; + accessor->read_bytes = STS_CHUNK_HEADER_SIZE; + + /* Go around again, so we can get a tuple from this chunk. */ + } else { + if (accessor->read_file != NULL) { + BufFileClose(accessor->read_file); + accessor->read_file = NULL; + } + + /* + * Try the next participant's file. If we've gone full circle, + * we're done. + */ + accessor->read_participant = (accessor->read_participant + 1) % accessor->sts->nparticipants; + if (accessor->read_participant == accessor->participant) { + break; + } + accessor->read_next_page = 0; + + /* Go around again, so we can get a chunk from this file. */ + } + } + + return NULL; +} + +/* + * Create the name used for the BufFile that a given participant will write. + */ +static void sts_filename(char* name, size_t length, SharedTuplestoreAccessor* accessor, int participant) +{ + error_t rc = snprintf_s(name, length, length - 1, "%s.p%d", accessor->sts->name, participant); + securec_check_ss(rc, "", ""); +} diff --git a/src/gausskernel/storage/lmgr/lwlock.cpp b/src/gausskernel/storage/lmgr/lwlock.cpp index f9ff35b3e..61305e244 100755 --- a/src/gausskernel/storage/lmgr/lwlock.cpp +++ b/src/gausskernel/storage/lmgr/lwlock.cpp @@ -156,6 +156,7 @@ static const char *BuiltinTrancheNames[] = { /* LWTRANCHE_GTT_CTL */ "GlobalTempTableControl", "PLdebugger", + "SharedTupleStore", "parallel_append" }; diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h index 610303859..65c174503 100755 --- a/src/include/storage/lwlock.h +++ b/src/include/storage/lwlock.h @@ -143,8 +143,8 @@ enum BuiltinTrancheIds { LWTRANCHE_EXTEND, // For general 3rd plugin LWTRANCHE_GTT_CTL, // For GTT LWTRANCHE_PLDEBUG, // For Pldebugger + LWTRANCHE_SHARED_TUPLESTORE, LWTRANCHE_PARALLEL_APPEND, - /* * Each trancheId above should have a corresponding item in BuiltinTrancheNames; */ diff --git a/src/include/utils/sharedtuplestore.h b/src/include/utils/sharedtuplestore.h new file mode 100644 index 000000000..14c6e6b06 --- /dev/null +++ b/src/include/utils/sharedtuplestore.h @@ -0,0 +1,51 @@ +/*------------------------------------------------------------------------- + * + * sharedtuplestore.h + * Simple mechanism for sharing tuples between backends. + * + * Portions Copyright (c) 1996-2020, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * src/include/utils/sharedtuplestore.h + * + *------------------------------------------------------------------------- + */ +#ifndef SHAREDTUPLESTORE_H +#define SHAREDTUPLESTORE_H + +#include "access/htup.h" +#include "storage/fd.h" +#include "storage/sharedfileset.h" + +struct SharedTuplestore; +typedef struct SharedTuplestore SharedTuplestore; + +struct SharedTuplestoreAccessor; +typedef struct SharedTuplestoreAccessor SharedTuplestoreAccessor; + +/* + * A flag indicating that the tuplestore will only be scanned once, so backing + * files can be unlinked early. + */ +#define SHARED_TUPLESTORE_SINGLE_PASS 0x01 + +extern size_t sts_estimate(int participants); + +extern SharedTuplestoreAccessor* sts_initialize(SharedTuplestore* sts, int participants, int my_participant_number, + size_t meta_data_size, int flags, SharedFileSet* fileset, const char* name); + +extern SharedTuplestoreAccessor* sts_attach(SharedTuplestore* sts, int my_participant_number, SharedFileSet* fileset); + +extern void sts_end_write(SharedTuplestoreAccessor* accessor); + +extern void sts_reinitialize(SharedTuplestoreAccessor* accessor); + +extern void sts_begin_parallel_scan(SharedTuplestoreAccessor* accessor); + +extern void sts_end_parallel_scan(SharedTuplestoreAccessor* accessor); + +extern void sts_puttuple(SharedTuplestoreAccessor* accessor, void* meta_data, MinimalTuple tuple); + +extern MinimalTuple sts_parallel_scan_next(SharedTuplestoreAccessor* accessor, void* meta_data); + +#endif /* SHAREDTUPLESTORE_H */ diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list index 7c463522f..a8967c590 100644 --- a/src/tools/pgindent/typedefs.list +++ b/src/tools/pgindent/typedefs.list @@ -1558,6 +1558,8 @@ SharedInvalRelcacheMsg SharedInvalRelmapMsg SharedInvalSmgrMsg SharedInvalidationMessage +SharedTuplestore +SharedTuplestoreAccessor ShellTypeInfo ShmemIndexEnt ShutdownInformation From 974b231240d78c1c6deb97752db432ecec88c043 Mon Sep 17 00:00:00 2001 From: wuyuechuan Date: Tue, 24 Nov 2020 11:08:59 +0800 Subject: [PATCH 2/2] parallel hash: inner parallel --- src/bin/gs_guc/cluster_guc.conf | 1 + src/common/backend/nodes/copyfuncs.cpp | 2 + src/common/backend/nodes/outfuncs.cpp | 2 + src/common/backend/nodes/readfuncs.cpp | 2 + src/common/backend/utils/misc/guc.cpp | 28 + .../backend/utils/misc/postgresql.conf.sample | 2 + .../backend/utils/sort/sharedtuplestore.cpp | 11 +- .../cbb/utils/partition/partitionmap.cpp | 13 + .../optimizer/commands/explain.cpp | 63 +- src/gausskernel/optimizer/path/allpaths.cpp | 16 +- src/gausskernel/optimizer/path/costsize.cpp | 25 +- src/gausskernel/optimizer/path/joinpath.cpp | 56 +- src/gausskernel/optimizer/plan/createplan.cpp | 27 +- .../optimizer/plan/streamplan_single.cpp | 2 +- src/gausskernel/optimizer/plan/subselect.cpp | 73 +- src/gausskernel/optimizer/util/pathnode.cpp | 81 +- src/gausskernel/optimizer/util/pruning.cpp | 44 +- .../runtime/executor/execParallel.cpp | 137 +- .../runtime/executor/execProcnode.cpp | 17 +- .../runtime/executor/nodeAppend.cpp | 14 + .../runtime/executor/nodeBitmapHeapscan.cpp | 27 +- .../runtime/executor/nodeBitmapIndexscan.cpp | 64 +- .../runtime/executor/nodeGather.cpp | 47 +- src/gausskernel/runtime/executor/nodeHash.cpp | 1657 +++++++++++++++-- .../runtime/executor/nodeHashjoin.cpp | 699 ++++++- .../runtime/executor/nodeIndexonlyscan.cpp | 62 +- .../runtime/executor/nodeIndexscan.cpp | 92 +- .../runtime/executor/nodeSeqscan.cpp | 46 +- src/gausskernel/runtime/executor/opfusion.cpp | 249 ++- .../runtime/executor/opfusion_scan.cpp | 70 +- .../runtime/executor/opfusion_util.cpp | 126 +- .../storage/access/heap/heapam.cpp | 26 +- .../storage/access/nbtree/nbtsearch.cpp | 2 +- src/gausskernel/storage/file/buffile.cpp | 1 + src/gausskernel/storage/ipc/Makefile | 2 +- src/gausskernel/storage/ipc/pg_barrier.cpp | 316 ++++ src/gausskernel/storage/lmgr/lwlock.cpp | 3 +- src/include/access/heapam.h | 1 + src/include/executor/execParallel.h | 2 +- src/include/executor/hashjoin.h | 172 +- src/include/executor/nodeHash.h | 24 +- src/include/executor/nodeHashjoin.h | 6 + src/include/executor/nodeIndexscan.h | 1 + src/include/executor/nodeSeqscan.h | 1 + .../knl/knl_guc/knl_session_attr_sql.h | 2 + src/include/knl/knl_session.h | 24 +- src/include/knl/knl_thread.h | 1 + src/include/nodes/execnodes.h | 19 + src/include/nodes/plannodes.h | 8 +- src/include/nodes/relation.h | 8 +- src/include/opfusion/opfusion_scan.h | 4 + src/include/opfusion/opfusion_util.h | 9 +- src/include/optimizer/cost.h | 3 +- src/include/optimizer/pathnode.h | 3 +- src/include/storage/lwlock.h | 1 + src/include/storage/pg_barrier.h | 53 + src/include/utils/dsa.h | 82 + src/include/utils/partitionmap.h | 1 + src/include/utils/partitionmap_gs.h | 2 + src/include/utils/sharedtuplestore.h | 2 +- .../regress/expected/llt_coverage_atc.out | 8 +- src/test/regress/expected/parallel_append.out | 534 +++--- .../regress/expected/parallel_hashjoin.out | 33 +- .../regress/expected/parallel_nested_loop.out | 2 + src/test/regress/expected/parallel_query.out | 50 +- src/test/regress/expected/rangefuncs.out | 4 +- src/test/regress/parallel_schedule | 7 +- src/test/regress/parallel_schedule3 | 2 +- src/test/regress/sql/parallel_append.sql | 38 +- src/test/regress/sql/parallel_hashjoin.sql | 6 +- src/test/regress/sql/parallel_nested_loop.sql | 4 +- src/test/regress/sql/parallel_query.sql | 25 +- 72 files changed, 4423 insertions(+), 824 deletions(-) create mode 100644 src/gausskernel/storage/ipc/pg_barrier.cpp create mode 100644 src/include/storage/pg_barrier.h create mode 100644 src/include/utils/dsa.h diff --git a/src/bin/gs_guc/cluster_guc.conf b/src/bin/gs_guc/cluster_guc.conf index 3093ef098..baf9e9a7c 100644 --- a/src/bin/gs_guc/cluster_guc.conf +++ b/src/bin/gs_guc/cluster_guc.conf @@ -192,6 +192,7 @@ enable_online_ddl_waitlock|bool|0,0|NULL|It is not recommended to enable this pa enable_user_metric_persistent|bool|0,0|NULL|NULL| enable_opfusion|bool|0,0|NULL|NULL| enable_parallel_append|bool|0,0|NULL|NULL| +enable_parallel_hash|bool|0,0|NULL|NULL| enable_partitionwise|bool|0,0|NULL|NULL| enable_pbe_optimization|bool|0,0|NULL|NULL| enable_prevent_job_task_startup|bool|0,0|NULL|It is not recommended to enable this parameter except for scaling out.| diff --git a/src/common/backend/nodes/copyfuncs.cpp b/src/common/backend/nodes/copyfuncs.cpp index f22dfbd99..502fbe6e0 100644 --- a/src/common/backend/nodes/copyfuncs.cpp +++ b/src/common/backend/nodes/copyfuncs.cpp @@ -441,6 +441,7 @@ static Gather *_copyGather(const Gather *from) * copy remainder of node */ COPY_SCALAR_FIELD(num_workers); + COPY_SCALAR_FIELD(rescan_param); COPY_SCALAR_FIELD(single_copy); return newnode; @@ -1404,6 +1405,7 @@ static Hash* _copyHash(const Hash* from) COPY_SCALAR_FIELD(skewTable); COPY_SCALAR_FIELD(skewColumn); COPY_SCALAR_FIELD(skewInherit); + COPY_SCALAR_FIELD(rows_total); COPY_SCALAR_FIELD(skewColType); COPY_SCALAR_FIELD(skewColTypmod); diff --git a/src/common/backend/nodes/outfuncs.cpp b/src/common/backend/nodes/outfuncs.cpp index ff4ad7c0e..4d354008e 100755 --- a/src/common/backend/nodes/outfuncs.cpp +++ b/src/common/backend/nodes/outfuncs.cpp @@ -908,6 +908,7 @@ static void _outGather(StringInfo str, Gather *node) _outPlanInfo(str, (Plan *)node); WRITE_INT_FIELD(num_workers); + WRITE_INT_FIELD(rescan_param); WRITE_BOOL_FIELD(single_copy); } @@ -1737,6 +1738,7 @@ static void _outHash(StringInfo str, Hash* node) WRITE_INT_FIELD(skewColumn); WRITE_BOOL_FIELD(skewInherit); + WRITE_FLOAT_FIELD(rows_total, "%.0f"); WRITE_OID_FIELD(skewColType); WRITE_INT_FIELD(skewColTypmod); WRITE_TYPEINFO_FIELD(skewColType); diff --git a/src/common/backend/nodes/readfuncs.cpp b/src/common/backend/nodes/readfuncs.cpp index 938c6cf3a..54da11848 100644 --- a/src/common/backend/nodes/readfuncs.cpp +++ b/src/common/backend/nodes/readfuncs.cpp @@ -3574,6 +3574,7 @@ static Hash* _readHash(Hash* local_node) READ_INT_FIELD(skewColumn); READ_BOOL_FIELD(skewInherit); + READ_FLOAT_FIELD(rows_total); READ_OID_FIELD(skewColType); READ_INT_FIELD(skewColTypmod); @@ -4870,6 +4871,7 @@ static Gather* _readGather(void) _readPlan(&local_node->plan); READ_INT_FIELD(num_workers); + READ_INT_FIELD(rescan_param); READ_BOOL_FIELD(single_copy); READ_DONE(); } diff --git a/src/common/backend/utils/misc/guc.cpp b/src/common/backend/utils/misc/guc.cpp index 297e61e38..4d392b439 100644 --- a/src/common/backend/utils/misc/guc.cpp +++ b/src/common/backend/utils/misc/guc.cpp @@ -1672,6 +1672,20 @@ static void init_configure_names_bool() NULL, NULL }, + { + { + "enable_parallel_hash", + PGC_USERSET, + QUERY_TUNING_METHOD, + gettext_noop("Enables the planner's user of parallel hash plans."), + NULL + }, + &u_sess->attr.attr_sql.enable_parallel_hash, + true, + NULL, + NULL, + NULL + }, { { "enable_nodegroup_debug", @@ -4489,6 +4503,20 @@ static void init_configure_names_bool() NULL, NULL }, + { + { + "enable_partition_opfusion", + PGC_USERSET, + QUERY_TUNING_METHOD, + gettext_noop("Enables opfusion features in partition table."), + NULL + }, + &u_sess->attr.attr_sql.enable_partition_opfusion, + false, + NULL, + NULL, + NULL + }, { { "check_implicit_conversions", diff --git a/src/common/backend/utils/misc/postgresql.conf.sample b/src/common/backend/utils/misc/postgresql.conf.sample index 748d515af..516eb4b4f 100755 --- a/src/common/backend/utils/misc/postgresql.conf.sample +++ b/src/common/backend/utils/misc/postgresql.conf.sample @@ -334,6 +334,8 @@ hot_standby = on # "on" allows queries during recovery #enable_seqscan = on #enable_sort = on #enable_tidscan = on +#enable_parallel_append = on +#enable_parallel_hash = on enable_kill_query = off # optional: [on, off], default: off #enforce_a_behavior = on # - Planner Cost Constants - diff --git a/src/common/backend/utils/sort/sharedtuplestore.cpp b/src/common/backend/utils/sort/sharedtuplestore.cpp index 4ec3ea8d4..73a457bb4 100644 --- a/src/common/backend/utils/sort/sharedtuplestore.cpp +++ b/src/common/backend/utils/sort/sharedtuplestore.cpp @@ -1,6 +1,6 @@ /*------------------------------------------------------------------------- * - * sharedtuplestore.c + * sharedtuplestore.cpp * Simple mechanism for sharing tuples between backends. * * This module contains a shared temporary tuple storage mechanism providing @@ -12,9 +12,10 @@ * * Portions Copyright (c) 1996-2020, PostgreSQL Global Development Group * Portions Copyright (c) 1994, Regents of the University of California + * Portions Copyright (c) 2020 Huawei Technologies Co.,Ltd. * * IDENTIFICATION - * src/backend/utils/sort/sharedtuplestore.c + * src/common/backend/utils/sort/sharedtuplestore.cpp * *------------------------------------------------------------------------- */ @@ -182,7 +183,7 @@ static void sts_flush_chunk(SharedTuplestoreAccessor* accessor) size_t size; size = WRITE_CHUNK_LEN; - BufFileWrite(accessor->write_file, accessor->write_chunk, size); + (void)BufFileWrite(accessor->write_file, accessor->write_chunk, size); int rc = memset_s(accessor->write_chunk, size, 0, size); securec_check(rc, "", ""); @@ -275,7 +276,7 @@ void sts_end_parallel_scan(SharedTuplestoreAccessor* accessor) * Write a tuple. If a meta-data size was provided to sts_initialize, then a * pointer to meta data of that size must be provided. */ -void sts_puttuple(SharedTuplestoreAccessor* accessor, void* meta_data, MinimalTuple tuple) +void sts_puttuple(SharedTuplestoreAccessor* accessor, const void* meta_data, MinimalTuple tuple) { size_t size; errno_t rc; @@ -494,7 +495,7 @@ MinimalTuple sts_parallel_scan_next(SharedTuplestoreAccessor* accessor, void* me /* Find the location of a new chunk to read. */ p = &accessor->sts->participants[accessor->read_participant]; - LWLockAcquire(&p->lock, LW_EXCLUSIVE); + (void)LWLockAcquire(&p->lock, LW_EXCLUSIVE); /* We can skip directly past overflow pages we know about. */ if (p->read_page < accessor->read_next_page) { p->read_page = accessor->read_next_page; diff --git a/src/gausskernel/cbb/utils/partition/partitionmap.cpp b/src/gausskernel/cbb/utils/partition/partitionmap.cpp index 71ed65cf5..b15f984d2 100644 --- a/src/gausskernel/cbb/utils/partition/partitionmap.cpp +++ b/src/gausskernel/cbb/utils/partition/partitionmap.cpp @@ -1158,6 +1158,19 @@ Oid getRangePartitionOid(Relation relation, Const** partKeyValue, int32* partSeq return result; } +Oid getPartitionOidByParam(Relation relation, Param *paramArg, ParamExternData *prm) +{ + int16 typLen; + bool typByVal = false; + + Assert(prm->ptype == paramArg->paramtype); + get_typlenbyval(paramArg->paramtype, &typLen, &typByVal); + Const *value = makeConst(paramArg->paramtype, paramArg->paramtypmod, paramArg->paramcollid, + (int)typLen, prm->value, prm->isnull, typByVal); + + return getRangePartitionOid(relation, &value, NULL, true); +} + inline Const* CalcLowBoundary(const Const* upBoundary, Interval* intervalValue) { Assert(upBoundary->consttype == TIMESTAMPOID || upBoundary->consttype == TIMESTAMPTZOID || diff --git a/src/gausskernel/optimizer/commands/explain.cpp b/src/gausskernel/optimizer/commands/explain.cpp index 9db89b9d1..ddb212a5c 100755 --- a/src/gausskernel/optimizer/commands/explain.cpp +++ b/src/gausskernel/optimizer/commands/explain.cpp @@ -1497,6 +1497,13 @@ static void show_pruning_info(PlanState* planstate, ExplainState* es, bool is_pr appendStringInfo(es->str, "NONE"); else appendStringInfo(es->planinfo->m_detailInfo->info_str, "NONE"); + } else if (scanplan->pruningInfo->paramArg != NULL) { + if (is_pretty == false) { + appendStringInfo(es->str, "$%d", scanplan->pruningInfo->paramArg->paramid); + } else { + appendStringInfo(es->planinfo->m_detailInfo->info_str, "$%d", + scanplan->pruningInfo->paramArg->paramid); + } } else { ListCell* cell = NULL; List* part_seqs = scanplan->pruningInfo->ls_rangeSelectedPartitions; @@ -1586,6 +1593,12 @@ static void show_pruning_info(PlanState* planstate, ExplainState* es, bool is_pr } else { if (scanplan->itrs <= 0) { ExplainPropertyText("Selected Partitions", "NONE", es); + } else if (scanplan->pruningInfo->paramArg != NULL) { + StringInfo strif = makeStringInfo(); + appendStringInfo(strif, "$%d", scanplan->pruningInfo->paramArg->paramid); + ExplainPropertyText("Selected Partitions", strif->data, es); + pfree(strif->data); + pfree(strif); } else { int i = 0; StringInfo strif; @@ -4583,18 +4596,46 @@ static void show_hash_info(HashState* hashstate, ExplainState* es) } } } - } else if (hashstate->ps.instrument) { - SortHashInfo hashinfo = hashstate->ps.instrument->sorthashinfo; - spacePeakKb = (hashinfo.spacePeak + BYTE_PER_KB - 1) / BYTE_PER_KB; - nbatch = hashinfo.nbatch; - nbatch_original = hashinfo.nbatch_original; - nbuckets = hashinfo.nbuckets; + } else { + Instrumentation* instrument = NULL; + /* + * In a parallel query, the leader process may or may not have run the + * hash join, and even if it did it may not have built a hash table due to + * timing (if it started late it might have seen no tuples in the outer + * relation and skipped building the hash table). Therefore we have to be + * prepared to get instrumentation data from a worker if there is no hash + * table. + */ + if (hashstate->hashtable) { + instrument = (Instrumentation*)palloc(sizeof(Instrumentation)); + ExecHashGetInstrumentation(instrument, hashstate->hashtable); + } else if (hashstate->shared_info) { + SharedHashInfo* shared_info = hashstate->shared_info; + int i; - /* wlm_statistics_plan_max_digit: this variable is used to judge, isn't it a active sql */ - if (es->wlm_statistics_plan_max_digit == NULL) { - if (es->format == EXPLAIN_FORMAT_TEXT) - appendStringInfoSpaces(es->str, es->indent * 2); - show_datanode_hash_info(es, nbatch, nbatch_original, nbuckets, spacePeakKb); + /* Find the first worker that built a hash table. */ + for (i = 0; i < shared_info->num_workers; ++i) { + if (shared_info->instrument[i].sorthashinfo.nbatch > 0) { + instrument = &shared_info->instrument[i]; + break; + } + } + } else if (hashstate->ps.instrument) { + instrument = hashstate->ps.instrument; + } + if (instrument) { + SortHashInfo hashinfo = instrument->sorthashinfo; + spacePeakKb = (hashinfo.spacePeak + BYTE_PER_KB - 1) / BYTE_PER_KB; + nbatch = hashinfo.nbatch; + nbatch_original = hashinfo.nbatch_original; + nbuckets = hashinfo.nbuckets; + + /* wlm_statistics_plan_max_digit: this variable is used to judge, isn't it a active sql */ + if (es->wlm_statistics_plan_max_digit == NULL) { + if (es->format == EXPLAIN_FORMAT_TEXT) + appendStringInfoSpaces(es->str, es->indent * 2); + show_datanode_hash_info(es, nbatch, nbatch_original, nbuckets, spacePeakKb); + } } } } diff --git a/src/gausskernel/optimizer/path/allpaths.cpp b/src/gausskernel/optimizer/path/allpaths.cpp index 2216d658c..9c30037e5 100755 --- a/src/gausskernel/optimizer/path/allpaths.cpp +++ b/src/gausskernel/optimizer/path/allpaths.cpp @@ -667,8 +667,8 @@ static void set_rel_pathlist(PlannerInfo* root, RelOptInfo* rel, Index rti, Rang generate_gather_paths(root, rel); } - /* - * Find the cheapest of the paths for this rel here because + /* + * Find the cheapest of the paths for this rel here because * generate_gather_paths may delete a path that some paths have * a reference to. */ @@ -715,9 +715,13 @@ static void set_plain_rel_size(PlannerInfo* root, RelOptInfo* rel, RangeTblEntry Assert(rel->pruning_result); - /* set flag for dealing with partintioned table */ - rel->partItrs = bms_num_members(rel->pruning_result->bm_rangeSelectedPartitions) + - bms_num_members(rel->pruning_result->intervalSelectedPartitions); + if (u_sess->attr.attr_sql.enable_pbe_optimization && rel->pruning_result->paramArg != NULL) { + rel->partItrs = 1; + } else { + /* set flag for dealing with partintioned table */ + rel->partItrs = bms_num_members(rel->pruning_result->bm_rangeSelectedPartitions) + + bms_num_members(rel->pruning_result->intervalSelectedPartitions); + } if (relation->partMap != NULL && PartitionMapIsRange(relation->partMap)) { RangePartitionMap *partMmap = (RangePartitionMap *)relation->partMap; @@ -1783,7 +1787,7 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel, List *li parallel_workers = Max(parallel_workers, path->parallel_workers); } - + /* * Same formula here as above. It's even more important in this * instance because the non-partial paths won't contribute anything to diff --git a/src/gausskernel/optimizer/path/costsize.cpp b/src/gausskernel/optimizer/path/costsize.cpp index 1b47a83eb..4b34e3713 100644 --- a/src/gausskernel/optimizer/path/costsize.cpp +++ b/src/gausskernel/optimizer/path/costsize.cpp @@ -3602,12 +3602,14 @@ MergeScanSelCache* cached_scansel(PlannerInfo* root, RestrictInfo* rinfo, PathKe * 'semifactors' contains valid data if jointype is SEMI or ANTI */ void initial_cost_hashjoin(PlannerInfo* root, JoinCostWorkspace* workspace, JoinType jointype, List* hashclauses, - Path* outer_path, Path* inner_path, SpecialJoinInfo* sjinfo, SemiAntiJoinFactors* semifactors, int dop) + Path* outer_path, Path* inner_path, SpecialJoinInfo* sjinfo, SemiAntiJoinFactors* semifactors, int dop, + bool parallel_hash) { Cost startup_cost = 0; Cost run_cost = 0; double outer_path_rows = PATH_LOCAL_ROWS(outer_path) / dop; double inner_path_rows = PATH_LOCAL_ROWS(inner_path) / dop; + double inner_path_rows_total = inner_path_rows; int num_hashclauses = list_length(hashclauses); int numbuckets; int numbatches; @@ -3616,6 +3618,7 @@ void initial_cost_hashjoin(PlannerInfo* root, JoinCostWorkspace* workspace, Join int outer_width; /* width of outer rel */ double outerpages; double innerpages; + size_t space_allowed; /* unused */ errno_t rc = 0; @@ -3630,6 +3633,16 @@ void initial_cost_hashjoin(PlannerInfo* root, JoinCostWorkspace* workspace, Join /* cost of source data */ startup_cost += outer_path->startup_cost; run_cost += outer_path->total_cost - outer_path->startup_cost; + + /* + * If this is a parallel hash build, then the value we have for + * inner_rows_total currently refers only to the rows returned by each + * participant. For shared hash table size estimation, we need the total + * number, so we need to undo the division. + */ + if (u_sess->attr.attr_sql.enable_parallel_hash) { + inner_path_rows_total *= get_parallel_divisor(inner_path); + } /* * Sometimes, we suffers the case that small table with large cost join * with a large table. In such case, the cost mainly comes from large cost @@ -3690,9 +3703,12 @@ void initial_cost_hashjoin(PlannerInfo* root, JoinCostWorkspace* workspace, Join */ inner_width = get_path_actual_total_width(inner_path, root->glob->vectorized, OP_HASHJOIN, newcolnum); outer_width = get_path_actual_total_width(outer_path, root->glob->vectorized, OP_HASHJOIN); - ExecChooseHashTableSize(inner_path_rows, + ExecChooseHashTableSize(inner_path_rows_total, inner_width, true, + parallel_hash, /* try_combined_work_mem */ + outer_path->parallel_workers, + &space_allowed, &numbuckets, &numbatches, &num_skew_mcvs, @@ -3759,6 +3775,7 @@ void initial_cost_hashjoin(PlannerInfo* root, JoinCostWorkspace* workspace, Join workspace->run_cost = run_cost; workspace->numbuckets = numbuckets; workspace->numbatches = numbatches; + workspace->inner_rows_total = inner_path_rows_total; ereport(DEBUG1, (errmodule(MOD_OPT_JOIN), @@ -3872,6 +3889,7 @@ void final_cost_hashjoin(PlannerInfo* root, HashPath* path, JoinCostWorkspace* w Path* inner_path = path->jpath.innerjoinpath; double outer_path_rows = PATH_LOCAL_ROWS(outer_path) / dop; double inner_path_rows = PATH_LOCAL_ROWS(inner_path) / dop; + double inner_path_rows_total = workspace->inner_rows_total; List* hashclauses = path->path_hashclauses; Cost startup_cost = workspace->startup_cost; Cost run_cost = workspace->run_cost; @@ -3912,6 +3930,9 @@ void final_cost_hashjoin(PlannerInfo* root, HashPath* path, JoinCostWorkspace* w /* mark the path with estimated # of batches */ path->num_batches = numbatches; + /* store the total number of tuples (sum of partial row estimates) */ + path->inner_rows_total = inner_path_rows_total; + /* and compute the number of "virtual" buckets in the whole join */ virtualbuckets = (double)numbuckets * (double)numbatches; diff --git a/src/gausskernel/optimizer/path/joinpath.cpp b/src/gausskernel/optimizer/path/joinpath.cpp index edcd53ca8..3c119f0bf 100755 --- a/src/gausskernel/optimizer/path/joinpath.cpp +++ b/src/gausskernel/optimizer/path/joinpath.cpp @@ -825,7 +825,7 @@ static void try_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType j ? 1 : u_sess->opt_cxt.query_dop; initial_cost_hashjoin( - root, &workspace, jointype, hashclauses, outer_path, inner_path, sjinfo, semifactors, max_dop); + root, &workspace, jointype, hashclauses, outer_path, inner_path, sjinfo, semifactors, max_dop, false); if (add_path_precheck(joinrel, workspace.startup_cost, workspace.total_cost, NIL, required_outer) || add_path_hintcheck(root->parse->hintState, joinrel->relids, outer_path, inner_path, HINT_KEYWORD_HASHJOIN)) { @@ -899,6 +899,7 @@ static void try_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType j semifactors, outerpath, innerpath, + false, /* parallel_hash */ restrict_clauses, required_outer, hashclauses); @@ -914,6 +915,7 @@ static void try_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType j semifactors, outer_path, inner_path, + false, /* parallel_hash */ restrict_clauses, required_outer, hashclauses)); @@ -929,9 +931,13 @@ static void try_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType j * try_partial_hashjoin_path * Consider a partial hashjoin join path; if it appears useful, push it into * the joinrel's partial_pathlist via add_partial_path(). + * The outer side is partial. If parallel_hash is true, then the inner path + * must be partial and will be run in parallel to create one or more shared + * hash tables; otherwise the inner path must be complete and a copy of it + * is run in every process to create separate identical private hash tables. */ static void try_partial_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, Path* outer_path, Path* inner_path, - List* hashclauses, JoinType jointype, JoinPathExtraData* extra) + List* hashclauses, JoinType jointype, JoinPathExtraData* extra, bool parallel_hash) { JoinCostWorkspace workspace; @@ -953,8 +959,16 @@ static void try_partial_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, Pa * Before creating a path, get a quick lower bound on what it is likely * to cost. Bail out right away if it looks terrible. */ - initial_cost_hashjoin( - root, &workspace, jointype, hashclauses, outer_path, inner_path, extra->sjinfo, &extra->semifactors, 1); + initial_cost_hashjoin(root, + &workspace, + jointype, + hashclauses, + outer_path, + inner_path, + extra->sjinfo, + &extra->semifactors, + 1, + parallel_hash); if (!add_partial_path_precheck(joinrel, workspace.total_cost, NIL)) { return; } @@ -969,6 +983,7 @@ static void try_partial_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, Pa &extra->semifactors, outer_path, inner_path, + parallel_hash, extra->restrictlist, NULL, hashclauses)); @@ -1775,14 +1790,35 @@ static void hash_inner_and_outer(PlannerInfo* root, RelOptInfo* joinrel, RelOptI * because the outer path will be partial, and therefore we won't be * able to properly guarantee uniqueness. Also, the resulting path * must not be parameterized. + * We would be able to support JOIN_FULL and JOIN_RIGHT for Parallelgs + * Hash, since in that case we're back to a single hash table with a + * single set of match bits for each batch, but that will require + * figuring out a deadlock-free way to wait for the probe to finish. */ if (joinrel->consider_parallel && jointype != JOIN_UNIQUE_OUTER && jointype != JOIN_FULL && jointype != JOIN_RIGHT && outerrel->partial_pathlist != NIL) { - Path* cheapest_partial_outer; + Path* cheapest_partial_outer = NULL; + Path* cheapest_partial_inner = NULL; Path* cheapest_safe_inner = NULL; cheapest_partial_outer = (Path*)linitial(outerrel->partial_pathlist); + /* + * Can we use a partial inner plan too, so that we can build a + * shared hash table in parallel? + */ + if (innerrel->partial_pathlist != NIL && u_sess->attr.attr_sql.enable_parallel_hash) { + cheapest_partial_inner = (Path*)linitial(innerrel->partial_pathlist); + try_partial_hashjoin_path(root, + joinrel, + cheapest_partial_outer, + cheapest_partial_inner, + hashclauses, + jointype, + extra, + true /* parallel_hash */); + } + /* * Normally, given that the joinrel is parallel-safe, the cheapest * total inner path will also be parallel-safe, but if not, we'll @@ -1805,8 +1841,14 @@ static void hash_inner_and_outer(PlannerInfo* root, RelOptInfo* joinrel, RelOptI } if (cheapest_safe_inner != NULL) { - try_partial_hashjoin_path( - root, joinrel, cheapest_partial_outer, cheapest_safe_inner, hashclauses, jointype, extra); + try_partial_hashjoin_path(root, + joinrel, + cheapest_partial_outer, + cheapest_safe_inner, + hashclauses, + jointype, + extra, + false /* parallel_hash */); } } } diff --git a/src/gausskernel/optimizer/plan/createplan.cpp b/src/gausskernel/optimizer/plan/createplan.cpp index 1225a3a09..cc3862d60 100644 --- a/src/gausskernel/optimizer/plan/createplan.cpp +++ b/src/gausskernel/optimizer/plan/createplan.cpp @@ -131,7 +131,8 @@ static Plan* setPartitionParam(PlannerInfo* root, Plan* plan, RelOptInfo* rel); static Plan* setBucketInfoParam(PlannerInfo* root, Plan* plan, RelOptInfo* rel); Plan* create_globalpartInterator_plan(PlannerInfo* root, PartIteratorPath* pIterpath); -static Gather *make_gather(List *qptlist, List *qpqual, int nworkers, bool single_copy, Plan *subplan); +static Gather* make_gather( + List* qptlist, List* qpqual, int nworkers, int rescan_param, bool single_copy, Plan* subplan); static IndexScan* make_indexscan(List* qptlist, List* qpqual, Index scanrelid, Oid indexid, List* indexqual, List* indexqualorig, List* indexorderby, List* indexorderbyorig, ScanDirection indexscandir); static IndexOnlyScan* make_indexonlyscan(List* qptlist, List* qpqual, Index scanrelid, Oid indexid, List* indexqual, @@ -1836,8 +1837,12 @@ static Gather* create_gather_plan(PlannerInfo* root, GatherPath* best_path) disuse_physical_tlist(subplan, best_path->subpath); - Gather* gather_plan = - make_gather(subplan->targetlist, NIL, best_path->path.parallel_workers, best_path->single_copy, subplan); + Gather* gather_plan = make_gather(subplan->targetlist, + NIL, + best_path->path.parallel_workers, + SS_assign_special_param(root), + best_path->single_copy, + subplan); copy_path_costsize(&gather_plan->plan, &best_path->path); @@ -4684,6 +4689,15 @@ static HashJoin* create_hashjoin_plan(PlannerInfo* root, HashPath* best_path, Pl * Build the hash node and hash join node. */ hash_plan = make_hash(inner_plan, skewTable, skewColumn, skewInherit, skewColType, skewColTypmod); + /* + * If parallel-aware, the executor will also need an estimate of the total + * number of rows expected from all participants so that it can size the + * shared hash table. + */ + if (best_path->jpath.path.parallel_aware) { + hash_plan->plan.parallel_aware = true; + hash_plan->rows_total = best_path->inner_rows_total; + } join_plan = make_hashjoin( tlist, joinclauses, otherclauses, hashclauses, outer_plan, (Plan*)hash_plan, best_path->jpath.jointype); @@ -6210,6 +6224,7 @@ static void estimate_directHashjoin_Cost( int num_skew_mcvs; int inner_width = hash_plan->plan_width; /* width of inner rel */ int outer_width = outerPlan->plan_width; /* width of outer rel */ + size_t space_allowed; /* unused */ errno_t rc = 0; rc = memset_s(&inner_mem_info, sizeof(OpMemInfo), 0, sizeof(OpMemInfo)); @@ -6233,6 +6248,9 @@ static void estimate_directHashjoin_Cost( ExecChooseHashTableSize(inner_path_rows, inner_width, true, /* useskew */ + false, + 0, + &space_allowed, &numbuckets, &numbatches, &num_skew_mcvs, @@ -7589,7 +7607,7 @@ Unique* make_unique(Plan* lefttree, List* distinctList) return node; } -static Gather *make_gather(List *qptlist, List *qpqual, int nworkers, bool single_copy, Plan *subplan) +static Gather *make_gather(List *qptlist, List *qpqual, int nworkers, int rescan_param, bool single_copy, Plan *subplan) { Gather *node = makeNode(Gather); Plan *plan = &node->plan; @@ -7600,6 +7618,7 @@ static Gather *make_gather(List *qptlist, List *qpqual, int nworkers, bool singl plan->lefttree = subplan; plan->righttree = NULL; node->num_workers = nworkers; + node->rescan_param = rescan_param; node->single_copy = single_copy; return node; diff --git a/src/gausskernel/optimizer/plan/streamplan_single.cpp b/src/gausskernel/optimizer/plan/streamplan_single.cpp index 8764ea183..335f27d82 100755 --- a/src/gausskernel/optimizer/plan/streamplan_single.cpp +++ b/src/gausskernel/optimizer/plan/streamplan_single.cpp @@ -571,7 +571,7 @@ void finalize_node_id(Plan* result_plan, int* plan_node_id, int* parent_node_id, subplan_ids[subplanid] = subplan_ids[0]; *parent_node_id = save_parent_id; } - if (IsExplainPlanStmt) { + if (IsExplainPlanStmt || IS_SINGLE_NODE) { CteScan* cte_plan = (CteScan*)result_plan; Plan* ru_plan = (Plan*)list_nth(subplans, cte_plan->ctePlanId - 1); if (!IsA(ru_plan, RecursiveUnion)) diff --git a/src/gausskernel/optimizer/plan/subselect.cpp b/src/gausskernel/optimizer/plan/subselect.cpp index c6545e96b..600925a44 100755 --- a/src/gausskernel/optimizer/plan/subselect.cpp +++ b/src/gausskernel/optimizer/plan/subselect.cpp @@ -98,7 +98,8 @@ static bool simplify_EXISTS_query(Query* query); static Query* convert_EXISTS_to_ANY(PlannerInfo* root, Query* subselect, Node** testexpr, List** paramIds); static Node* replace_correlation_vars_mutator(Node* node, PlannerInfo* root); static Node* process_sublinks_mutator(Node* node, process_sublinks_context* context); -static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_params, Bitmapset* scan_params); +static Bitmapset* finalize_plan( + PlannerInfo* root, Plan* plan, int gather_param, Bitmapset* valid_params, Bitmapset* scan_params); static bool finalize_primnode(Node* node, finalize_primnode_context* context); static Node* convert_joinqual_to_antiqual(Node* node, Query* parse); static Node* convert_opexpr_to_boolexpr_for_antijoin(Node* node, Query* parse); @@ -2311,7 +2312,7 @@ void SS_finalize_plan(PlannerInfo* root, Plan* plan, bool attach_initplans) /* * Now recurse through plan tree. */ - (void)finalize_plan(root, plan, valid_params, NULL); + (void)finalize_plan(root, plan, -1, valid_params, NULL); bms_free_ext(valid_params); @@ -2371,20 +2372,23 @@ static bool finalize_agg_primnode(Node* node, finalize_primnode_context* context return expression_tree_walker(node, (bool (*)())finalize_agg_primnode, (void*)context); } -static void finalize_plans( - PlannerInfo* root, finalize_primnode_context* context, List* plans, Bitmapset* valid_params, Bitmapset* scan_params) +static void finalize_plans(PlannerInfo* root, finalize_primnode_context* context, List* plans, Bitmapset* valid_params, + Bitmapset* scan_params, int gather_param = -1) { ListCell* lc = NULL; foreach (lc, plans) { - context->paramids = - bms_add_members(context->paramids, finalize_plan(root, (Plan*)lfirst(lc), valid_params, scan_params)); + context->paramids = bms_add_members( + context->paramids, finalize_plan(root, (Plan*)lfirst(lc), gather_param, valid_params, scan_params)); } } /* * Recursive processing of all nodes in the plan tree * + * gather_param is the rescan_param of an ancestral Gather/GatherMerge, + * or -1 if there is none. + * * valid_params is the set of param IDs considered valid to reference in * this plan node or its children. * scan_params is a set of param IDs to force scan plan nodes to reference. @@ -2394,7 +2398,8 @@ static void finalize_plans( * The return value is the computed allParam set for the given Plan node. * This is just an internal notational convenience. */ -static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_params, Bitmapset* scan_params) +static Bitmapset* finalize_plan( + PlannerInfo* root, Plan* plan, int gather_param, Bitmapset* valid_params, Bitmapset* scan_params) { finalize_primnode_context context; int locally_added_param; @@ -2419,6 +2424,17 @@ static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_ (void)finalize_primnode((Node*)plan->targetlist, &context); (void)finalize_primnode((Node*)plan->qual, &context); + /* + * If it's a parallel-aware scan node, mark it as dependent on the parent + * Gather/GatherMerge's rescan Param. + */ + if (plan->parallel_aware) { + if (gather_param < 0) { + elog(ERROR, "parallel-aware plan node is not below a Gather"); + } + context.paramids = bms_add_member(context.paramids, gather_param); + } + /* Check additional node-type-specific fields */ switch (nodeTag(plan)) { case T_BaseResult: @@ -2573,7 +2589,7 @@ static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_ context.paramids = bms_add_members(context.paramids, scan_params); /* child nodes if any */ - finalize_plans(root, &context, cscan->extensible_plans, valid_params, scan_params); + finalize_plans(root, &context, cscan->extensible_plans, valid_params, scan_params, gather_param); } break; case T_ModifyTable: { @@ -2585,7 +2601,7 @@ static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_ scan_params = bms_add_member(bms_copy(scan_params), locally_added_param); (void)finalize_primnode((Node*)mtplan->returningLists, &context); (void)finalize_primnode((Node*)mtplan->updateTlist, &context); - finalize_plans(root, &context, mtplan->plans, valid_params, scan_params); + finalize_plans(root, &context, mtplan->plans, valid_params, scan_params, gather_param); } break; #ifdef PGXC case T_RemoteQuery: @@ -2594,27 +2610,28 @@ static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_ #endif case T_Append: { - finalize_plans(root, &context, ((Append*)plan)->appendplans, valid_params, scan_params); + finalize_plans(root, &context, ((Append*)plan)->appendplans, valid_params, scan_params, gather_param); } break; - case T_MergeAppend: { - finalize_plans(root, &context, ((MergeAppend*)plan)->mergeplans, valid_params, scan_params); + finalize_plans(root, &context, ((MergeAppend*)plan)->mergeplans, valid_params, scan_params, gather_param); } break; case T_BitmapAnd: { - finalize_plans(root, &context, ((BitmapAnd*)plan)->bitmapplans, valid_params, scan_params); + finalize_plans(root, &context, ((BitmapAnd*)plan)->bitmapplans, valid_params, scan_params, gather_param); } break; case T_BitmapOr: { - finalize_plans(root, &context, ((BitmapOr*)plan)->bitmapplans, valid_params, scan_params); + finalize_plans(root, &context, ((BitmapOr*)plan)->bitmapplans, valid_params, scan_params, gather_param); } break; case T_CStoreIndexAnd: { - finalize_plans(root, &context, ((CStoreIndexAnd*)plan)->bitmapplans, valid_params, scan_params); + finalize_plans( + root, &context, ((CStoreIndexAnd*)plan)->bitmapplans, valid_params, scan_params, gather_param); } break; case T_CStoreIndexOr: { - finalize_plans(root, &context, ((CStoreIndexOr*)plan)->bitmapplans, valid_params, scan_params); + finalize_plans( + root, &context, ((CStoreIndexOr*)plan)->bitmapplans, valid_params, scan_params, gather_param); } break; case T_NestLoop: { ListCell* l = NULL; @@ -2674,7 +2691,23 @@ static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_ valid_params = bms_add_member(bms_copy(valid_params), locally_added_param); /* wtParam does *not* get added to scan_params */ break; + case T_Gather: + /* child nodes are allowed to reference rescan_param, if any */ + locally_added_param = ((Gather*)plan)->rescan_param; + if (locally_added_param >= 0) { + valid_params = bms_add_member(bms_copy(valid_params), locally_added_param); + /* + * We currently don't support nested Gathers. The issue so + * far as this function is concerned would be how to identify + * which child nodes depend on which Gather. + */ + Assert(gather_param < 0); + /* Pass down rescan_param to child parallel-aware nodes */ + gather_param = locally_added_param; + } + /* rescan_param does *not* get added to scan_params */ + break; case T_LockRows: /* Force descendant scan nodes to reference epqParam */ locally_added_param = ((LockRows*)plan)->epqParam; @@ -2709,7 +2742,6 @@ static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_ case T_Material: case T_Sort: case T_Unique: - case T_Gather: case T_SetOp: case T_Group: case T_Stream: @@ -2724,18 +2756,19 @@ static Bitmapset* finalize_plan(PlannerInfo* root, Plan* plan, Bitmapset* valid_ } /* Process left and right child plans, if any */ - child_params = finalize_plan(root, plan->lefttree, valid_params, scan_params); + child_params = finalize_plan(root, plan->lefttree, gather_param, valid_params, scan_params); context.paramids = bms_add_members(context.paramids, child_params); if (nestloop_params != NULL) { /* right child can reference nestloop_params as well as valid_params */ - child_params = finalize_plan(root, plan->righttree, bms_union(nestloop_params, valid_params), scan_params); + child_params = + finalize_plan(root, plan->righttree, gather_param, bms_union(nestloop_params, valid_params), scan_params); /* ... and they don't count as parameters used at my level */ child_params = bms_difference(child_params, nestloop_params); bms_free_ext(nestloop_params); } else { /* easy case */ - child_params = finalize_plan(root, plan->righttree, valid_params, scan_params); + child_params = finalize_plan(root, plan->righttree, gather_param, valid_params, scan_params); } context.paramids = bms_add_members(context.paramids, child_params); diff --git a/src/gausskernel/optimizer/util/pathnode.cpp b/src/gausskernel/optimizer/util/pathnode.cpp index e059cc86c..ec04e5548 100755 --- a/src/gausskernel/optimizer/util/pathnode.cpp +++ b/src/gausskernel/optimizer/util/pathnode.cpp @@ -1656,8 +1656,8 @@ Path* create_seqscan_path(PlannerInfo* root, RelOptInfo* rel, Relids required_ou pathnode->parallel_workers = parallel_workers; #ifdef STREAMPLAN - /* - * We need to set locator_type for parallel query, cause we may send + /* + * We need to set locator_type for parallel query, cause we may send * this value to bg worker. If not, locator_type is the initial value '\0', * which make the later serialized plan truncated. */ @@ -2024,8 +2024,8 @@ IndexPath* create_index_path(PlannerInfo* root, IndexOptInfo* index, List* index pathnode->indexorderbycols = indexorderbycols; pathnode->indexscandir = indexscandir; #ifdef STREAMPLAN - /* - * We need to set locator_type for parallel query, cause we may send + /* + * We need to set locator_type for parallel query, cause we may send * this value to bg worker. If not, locator_type is the initial value '\0', * which make the later serialized plan truncated. */ @@ -3864,7 +3864,7 @@ MergePath* create_mergejoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinTyp * (this should be a subset of the restrict_clauses list) */ HashPath* create_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype, JoinCostWorkspace* workspace, - SpecialJoinInfo* sjinfo, SemiAntiJoinFactors* semifactors, Path* outer_path, Path* inner_path, + SpecialJoinInfo* sjinfo, SemiAntiJoinFactors* semifactors, Path* outer_path, Path* inner_path, bool parallel_hash, List* restrict_clauses, Relids required_outer, List* hashclauses, int dop) { HashPath* pathnode = makeNode(HashPath); @@ -3875,9 +3875,10 @@ HashPath* create_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType pathnode->jpath.path.param_info = get_joinrel_parampathinfo(root, joinrel, outer_path, inner_path, sjinfo, required_outer, &restrict_clauses); - pathnode->jpath.path.parallel_aware = false; + pathnode->jpath.path.parallel_aware = joinrel->consider_parallel && parallel_hash; pathnode->jpath.path.parallel_safe = joinrel->consider_parallel && outer_path->parallel_safe && inner_path->parallel_safe; + /* This is a foolish way to estimate parallel_workers, but for now... */ pathnode->jpath.path.parallel_workers = outer_path->parallel_workers; @@ -5881,8 +5882,16 @@ static JoinPath* add_join_redistribute_path(PlannerInfo* root, RelOptInfo* joinr /* Create join path. */ if (nodetag == T_HashJoin) { - initial_cost_hashjoin( - root, workspace, jointype, hashclauses, stream_path_outer, stream_path_inner, sjinfo, semifactors, joinDop); + initial_cost_hashjoin(root, + workspace, + jointype, + hashclauses, + stream_path_outer, + stream_path_inner, + sjinfo, + semifactors, + joinDop, + true); joinpath = (JoinPath*)create_hashjoin_path(root, joinrel, @@ -5892,6 +5901,7 @@ static JoinPath* add_join_redistribute_path(PlannerInfo* root, RelOptInfo* joinr semifactors, stream_path_outer, stream_path_inner, + true, restrictlist, required_outer, hashclauses, @@ -5998,7 +6008,7 @@ static void add_hashjoin_broadcast_path(PlannerInfo* root, RelOptInfo* joinrel, new_inner_path = stream_outer ? other_side : streamed_path; initial_cost_hashjoin( - root, workspace, jointype, hashclauses, new_outer_path, new_inner_path, sjinfo, semifactors, dop); + root, workspace, jointype, hashclauses, new_outer_path, new_inner_path, sjinfo, semifactors, dop, false); joinpath = (JoinPath*)create_hashjoin_path(root, joinrel, @@ -6008,6 +6018,7 @@ static void add_hashjoin_broadcast_path(PlannerInfo* root, RelOptInfo* joinrel, semifactors, new_outer_path, new_inner_path, + false, restrictlist, required_outer, hashclauses, @@ -6664,8 +6675,16 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype */ if (!parallel_enable(inner_path_t, outer_path_t)) { if (outer_path != outer_path_t || inner_path != inner_path_t) { - initial_cost_hashjoin( - root, workspace, jointype, hashclauses, outer_path_t, inner_path_t, sjinfo, semifactors, 1); + initial_cost_hashjoin(root, + workspace, + jointype, + hashclauses, + outer_path_t, + inner_path_t, + sjinfo, + semifactors, + 1, + false); } joinpath = (JoinPath*)create_hashjoin_path(root, @@ -6676,6 +6695,7 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype semifactors, outer_path_t, inner_path_t, + false, restrictlist, required_outer, hashclauses); @@ -6751,8 +6771,16 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype true, skew_stream, target_distribution); - initial_cost_hashjoin( - root, workspace, jointype, hashclauses, outer_path, stream_path_inner, sjinfo, semifactors, 1); + initial_cost_hashjoin(root, + workspace, + jointype, + hashclauses, + outer_path, + stream_path_inner, + sjinfo, + semifactors, + 1, + false); joinpath = (JoinPath*)create_hashjoin_path(root, joinrel, @@ -6762,6 +6790,7 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype semifactors, outer_path, (Path*)stream_path_inner, + false, restrictlist, required_outer, hashclauses); @@ -6901,8 +6930,16 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype skew_stream, target_distribution); - initial_cost_hashjoin( - root, workspace, jointype, hashclauses, stream_path_outer, inner_path, sjinfo, semifactors, 1); + initial_cost_hashjoin(root, + workspace, + jointype, + hashclauses, + stream_path_outer, + inner_path, + sjinfo, + semifactors, + 1, + false); joinpath = (JoinPath*)create_hashjoin_path(root, joinrel, @@ -6912,6 +6949,7 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype semifactors, (Path*)stream_path_outer, inner_path, + false, restrictlist, required_outer, hashclauses); @@ -7115,7 +7153,8 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype stream_path_inner, sjinfo, semifactors, - 1); + 1, + false); joinpath = (JoinPath*)create_hashjoin_path(root, joinrel, @@ -7125,6 +7164,7 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype semifactors, (Path*)stream_path_outer, (Path*)stream_path_inner, + false, restrictlist, required_outer, hashclauses); @@ -7284,7 +7324,8 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype stream_path_inner, sjinfo, semifactors, - 1); + 1, + false); joinpath = (JoinPath*)create_hashjoin_path(root, joinrel, @@ -7294,6 +7335,7 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype semifactors, outer_path, (Path*)stream_path_inner, + false, restrictlist, required_outer, hashclauses); @@ -7355,7 +7397,8 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype inner_path, sjinfo, semifactors, - 1); + 1, + false); joinpath = (JoinPath*)create_hashjoin_path(root, joinrel, @@ -7365,6 +7408,7 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype semifactors, (Path*)stream_path_outer, inner_path, + false, restrictlist, required_outer, hashclauses); @@ -7495,6 +7539,7 @@ void add_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype semifactors, outer_path, inner_path, + false, restrictlist, required_outer, hashclauses); diff --git a/src/gausskernel/optimizer/util/pruning.cpp b/src/gausskernel/optimizer/util/pruning.cpp index 3285ca9eb..460e1a48a 100755 --- a/src/gausskernel/optimizer/util/pruning.cpp +++ b/src/gausskernel/optimizer/util/pruning.cpp @@ -107,6 +107,7 @@ PruningResult* copyPruningResult(PruningResult* srcPruningResult) newpruningInfo->intervalOffset = srcPruningResult->intervalOffset; newpruningInfo->intervalSelectedPartitions = bms_copy(srcPruningResult->intervalSelectedPartitions); newpruningInfo->ls_rangeSelectedPartitions = list_copy(srcPruningResult->ls_rangeSelectedPartitions); + newpruningInfo->paramArg = (Param *)copyObject(srcPruningResult->paramArg); return newpruningInfo; } else { @@ -735,6 +736,7 @@ static PruningResult* intersectChildPruningResult(const List* resultList, Prunin result->bm_rangeSelectedPartitions = bms_copy(iteratorResult->bm_rangeSelectedPartitions); result->intervalSelectedPartitions = bms_copy(iteratorResult->intervalSelectedPartitions); result->state = iteratorResult->state; + result->paramArg = (Param *)copyObject(iteratorResult->paramArg); } else if (result != NULL) { if (intervalOffset == -1 && iteratorResult->intervalOffset >= 0) { intervalOffset = iteratorResult->intervalOffset; @@ -812,6 +814,10 @@ static PruningResult* unionChildPruningResult(const List* resultList, PruningCon if (iteratorResult->state == PRUNING_RESULT_EMPTY) { continue; + } else if (iteratorResult->paramArg != NULL) { + result->state = iteratorResult->state; + result->paramArg = (Param *)copyObject(iteratorResult->paramArg); + return result; } else if (iteratorResult->state == PRUNING_RESULT_FULL) { result->state = PRUNING_RESULT_FULL; return result; @@ -995,6 +1001,7 @@ static PruningResult* recordBoundaryFromOpExpr(const OpExpr* expr, PruningContex PruningResult* result = NULL; bool rightArgIsConst = true; Node* node = NULL; + Param* paramArg = NULL; AssertEreport(PointerIsValid(context), MOD_OPT, "Unexpected NULL pointer for context."); AssertEreport(PointerIsValid(context->relation), MOD_OPT, "Unexpected NULL pointer for context->relation."); @@ -1035,18 +1042,27 @@ static PruningResult* recordBoundaryFromOpExpr(const OpExpr* expr, PruningContex } /* one of args MUST be Const, and another argument Must be Var */ - if (!((T_Const == nodeTag(leftArg) && T_Var == nodeTag(rightArg)) || - (T_Var == nodeTag(leftArg) && T_Const == nodeTag(rightArg)))) { + /* Be const or param, for PBE */ + if (!(((T_Const == nodeTag(leftArg) || T_Param == nodeTag(leftArg)) && T_Var == nodeTag(rightArg)) || + (T_Var == nodeTag(leftArg) && (T_Const == nodeTag(rightArg) || T_Param == nodeTag(rightArg))))) { result->state = PRUNING_RESULT_FULL; return result; } - if (T_Const == nodeTag(leftArg)) { - constArg = (Const*)leftArg; + if (T_Var == nodeTag(rightArg)) { + if (T_Const == nodeTag(leftArg)) { + constArg = (Const*)leftArg; + } else { + paramArg = (Param *)leftArg; + } varArg = (Var*)rightArg; rightArgIsConst = false; } else { - constArg = (Const*)rightArg; + if (T_Const == nodeTag(rightArg)) { + constArg = (Const*)rightArg; + } else { + paramArg = (Param *)rightArg; + } varArg = (Var*)leftArg; } @@ -1067,6 +1083,19 @@ static PruningResult* recordBoundaryFromOpExpr(const OpExpr* expr, PruningContex return result; } + if (paramArg != NULL) { + /* Dynamic pruning for EXTERN params + range partition + '=' */ + if (!u_sess->attr.attr_sql.enable_pbe_optimization || paramArg->paramkind != PARAM_EXTERN + || strcmp("=", opName) != 0 || !PartitionMapIsRange(partMap)) { + result->state = PRUNING_RESULT_FULL; + return result; + } else { + result->paramArg = paramArg; + result->state = PRUNING_RESULT_SUBSET; + return result; + } + } + if (constArg->constisnull) { result->state = PRUNING_RESULT_EMPTY; return result; @@ -1749,6 +1778,11 @@ static void destroyPruningResult(PruningResult* pruningResult) pruningResult->ls_rangeSelectedPartitions = NIL; } + if (PointerIsValid(pruningResult->paramArg)) { + pfree(pruningResult->paramArg); + pruningResult->paramArg = NULL; + } + pfree_ext(pruningResult); } diff --git a/src/gausskernel/runtime/executor/execParallel.cpp b/src/gausskernel/runtime/executor/execParallel.cpp index 2c948e1ce..a51f16f2a 100644 --- a/src/gausskernel/runtime/executor/execParallel.cpp +++ b/src/gausskernel/runtime/executor/execParallel.cpp @@ -25,9 +25,12 @@ #include "executor/execParallel.h" #include "executor/executor.h" +#include "executor/hashjoin.h" #include "executor/nodeSeqscan.h" #include "executor/nodeAppend.h" #include "executor/nodeIndexscan.h" +#include "executor/nodeHashjoin.h" +#include "executor/nodeHash.h" #include "executor/tqueue.h" #include "nodes/nodeFuncs.h" #include "optimizer/planmain.h" @@ -69,6 +72,7 @@ typedef struct ExecParallelInitializeDSMContext { static char *ExecSerializePlan(Plan *plan, EState *estate); static bool ExecParallelEstimate(PlanState *node, ExecParallelEstimateContext *e); static bool ExecParallelInitializeDSM(PlanState *node, ExecParallelInitializeDSMContext *d); +static bool ExecParallelReInitializeDSM(PlanState* planstate, ParallelContext* pcxt); static shm_mq_handle **ExecParallelSetupTupleQueues(ParallelContext *pcxt, bool reinitialize); static bool ExecParallelRetrieveInstrumentation(PlanState *planstate, SharedExecutorInstrumentation *instrumentation); @@ -192,23 +196,38 @@ static bool ExecParallelInitializeDSM(PlanState *planstate, ExecParallelInitiali * estimated using shm_toc_allocate, and add the keys they previously * estimated using shm_toc_insert, in each case targeting pcxt->toc. */ - if (planstate->plan->parallel_aware) { - switch (nodeTag(planstate)) { - case T_SeqScanState: + switch (nodeTag(planstate)) { + case T_SeqScanState: + if (planstate->plan->parallel_aware) { ExecSeqScanInitializeDSM((SeqScanState *)planstate, d->pcxt, cxt->pwCtx->queryInfo.pscan_num); cxt->pwCtx->queryInfo.pscan_num++; - break; - case T_IndexScanState: + } + break; + case T_IndexScanState: + if (planstate->plan->parallel_aware) { ExecIndexScanInitializeDSM((IndexScanState*)planstate, d->pcxt, cxt->pwCtx->queryInfo.piscan_num); cxt->pwCtx->queryInfo.piscan_num++; - break; - case T_AppendState: + } + break; + case T_AppendState: + if (planstate->plan->parallel_aware) { ExecAppendInitializeDSM((AppendState *)planstate, d->pcxt, cxt->pwCtx->queryInfo.pappend_num); cxt->pwCtx->queryInfo.pappend_num++; - break; - default: - break; - } + } + break; + case T_HashJoinState: + if (planstate->plan->parallel_aware) { + ExecHashJoinInitializeDSM((HashJoinState*)planstate, d->pcxt, cxt->pwCtx->queryInfo.jstate_num); + cxt->pwCtx->queryInfo.jstate_num++; + } + break; + case T_HashState: + /* even when not parallel-aware, for EXPLAIN ANALYZE */ + ExecHashInitializeDSM((HashState*)planstate, d->pcxt, cxt->pwCtx->queryInfo.hash_num); + cxt->pwCtx->queryInfo.hash_num++; + break; + default: + break; } return planstate_tree_walker(planstate, (bool (*)())ExecParallelInitializeDSM, d); @@ -273,12 +292,13 @@ void ExecParallelCreateReaders(ParallelExecutorInfo *pei, TupleDesc tupDesc) * Re-initialize the parallel executor info such that it can be reused by * workers. */ -void ExecParallelReinitialize(ParallelExecutorInfo *pei) +void ExecParallelReinitialize(PlanState *planstate, ParallelExecutorInfo *pei) { ReinitializeParallelDSM(pei->pcxt); pei->tqueue = ExecParallelSetupTupleQueues(pei->pcxt, true); pei->reader = NULL; pei->finished = false; + (void)ExecParallelReInitializeDSM(planstate, pei->pcxt); } /* @@ -374,6 +394,8 @@ ParallelExecutorInfo *ExecInitParallelPlan(PlanState *planstate, EState *estate, queryInfo.pscan = (ParallelHeapScanDesc *)palloc0(sizeof(ParallelHeapScanDesc) * e.nnodes); queryInfo.piscan = (ParallelIndexScanDesc *)palloc0(sizeof(ParallelIndexScanDesc) * e.nnodes); queryInfo.pappend = (ParallelAppendState**)palloc0(sizeof(ParallelAppendState*) * e.nnodes); + queryInfo.jstate = (ParallelHashJoinState**)palloc0(sizeof(ParallelHashJoinState*) * e.nnodes); + queryInfo.shared_info = (SharedHashInfo**)palloc0(sizeof(SharedHashInfo*) * e.nnodes); /* * Give parallel-aware nodes a chance to initialize their shared data. @@ -405,6 +427,48 @@ ParallelExecutorInfo *ExecInitParallelPlan(PlanState *planstate, EState *estate, return pei; } +/* + * Traverse plan tree to reinitialize per-node dynamic shared memory state + */ +static bool ExecParallelReInitializeDSM(PlanState* planstate, ParallelContext* pcxt) +{ + if (planstate == NULL) { + return false; + } + /* + * Call reinitializers for DSM-using plan nodes. + */ + switch (nodeTag(planstate)) { + case T_SeqScanState: + if (planstate->plan->parallel_aware) { + ExecSeqScanReInitializeDSM((SeqScanState*)planstate, pcxt); + } + break; + case T_IndexScanState: + if (planstate->plan->parallel_aware) { + ExecIndexScanReInitializeDSM((IndexScanState*)planstate, pcxt); + } + break; + case T_AppendState: + if (planstate->plan->parallel_aware) { + ExecAppendReInitializeDSM((AppendState*)planstate, pcxt); + } + break; + case T_HashJoinState: + if (planstate->plan->parallel_aware) { + ExecHashJoinReInitializeDSM((HashJoinState*)planstate, pcxt); + } + break; + /* these nodes have DSM state, but no reinitialization is required */ + case T_HashState: + break; + default: + break; + } + + return planstate_tree_walker(planstate, (bool (*)())ExecParallelReInitializeDSM, pcxt); +} + /* * Copy instrumentation information about this node and its descendents from * dynamic shared memory. @@ -439,6 +503,14 @@ static bool ExecParallelRetrieveInstrumentation(PlanState *planstate, SharedExec int rc = memcpy_s(&planstate->worker_instrument->instrument, ibytes, instrument, ibytes); securec_check(rc, "", ""); + switch (nodeTag(planstate)) { + case T_HashState: + ExecHashRetrieveInstrumentation((HashState*)planstate); + break; + default: + break; + } + return planstate_tree_walker(planstate, (bool (*)())ExecParallelRetrieveInstrumentation, instrumentation); } @@ -603,20 +675,33 @@ static bool ExecParallelInitializeWorker(PlanState *planstate, void *context) return false; /* Call initializers for parallel-aware plan nodes. */ - if (planstate->plan->parallel_aware) { - switch (nodeTag(planstate)) { - case T_SeqScanState: - ExecSeqScanInitializeWorker((SeqScanState *)planstate, context); - break; - case T_IndexScanState: - ExecIndexScanInitializeWorker((IndexScanState *)planstate, context); - break; - case T_AppendState: - ExecAppendInitializeWorker((AppendState *)planstate, context); - break; - default: - break; - } + switch (nodeTag(planstate)) { + case T_SeqScanState: + if (planstate->plan->parallel_aware) { + ExecSeqScanInitializeWorker((SeqScanState*)planstate, context); + } + break; + case T_HashJoinState: + if (planstate->plan->parallel_aware) { + ExecHashJoinInitializeWorker((HashJoinState*)planstate, context); + } + break; + case T_IndexScanState: + if (planstate->plan->parallel_aware) { + ExecIndexScanInitializeWorker((IndexScanState*)planstate, context); + } + break; + case T_AppendState: + if (planstate->plan->parallel_aware) { + ExecAppendInitializeWorker((AppendState*)planstate, context); + } + break; + case T_HashState: + /* even when not parallel-aware, for EXPLAIN ANALYZE */ + ExecHashInitializeWorker((HashState*)planstate, context); + break; + default: + break; } return planstate_tree_walker(planstate, (bool (*)())ExecParallelInitializeWorker, context); diff --git a/src/gausskernel/runtime/executor/execProcnode.cpp b/src/gausskernel/runtime/executor/execProcnode.cpp index 80f52eae4..c016cc449 100755 --- a/src/gausskernel/runtime/executor/execProcnode.cpp +++ b/src/gausskernel/runtime/executor/execProcnode.cpp @@ -1360,20 +1360,27 @@ void ExecEndNode(PlanState* node) * no more rows will be needed (e.g. when a Limit is filled) rather than only * at the end of ExecutorRun. */ -bool ExecShutdownNode(PlanState *node) +bool ExecShutdownNode(PlanState* node) { - if (node == NULL) + if (node == NULL) { return false; - + } + (void)planstate_tree_walker(node, (bool (*)())ExecShutdownNode, NULL); switch (nodeTag(node)) { case T_GatherState: - ExecShutdownGather((GatherState *)node); + ExecShutdownGather((GatherState*)node); + break; + case T_HashState: + ExecShutdownHash((HashState*)node); + break; + case T_HashJoinState: + ExecShutdownHashJoin((HashJoinState*)node); break; default: break; } - return planstate_tree_walker(node, (bool (*)())ExecShutdownNode, NULL); + return false; } /* diff --git a/src/gausskernel/runtime/executor/nodeAppend.cpp b/src/gausskernel/runtime/executor/nodeAppend.cpp index d147c4e17..7514c4f89 100755 --- a/src/gausskernel/runtime/executor/nodeAppend.cpp +++ b/src/gausskernel/runtime/executor/nodeAppend.cpp @@ -277,6 +277,20 @@ void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt, int nodei node->choose_next_subplan = choose_next_subplan_for_leader; } +/* ---------------------------------------------------------------- + * ExecAppendReInitializeDSM + * + * Reset shared state before beginning a fresh scan. + * ---------------------------------------------------------------- + */ +void ExecAppendReInitializeDSM(AppendState* node, ParallelContext* pcxt) +{ + ParallelAppendState* pstate = node->as_pstate; + pstate->pa_next_plan = 0; + error_t rc = memset_s(pstate->pa_finished, sizeof(bool) * node->as_nplans, 0, sizeof(bool) * node->as_nplans); + securec_check(rc, "\0", "\0"); +} + /* ---------------------------------------------------------------- * ExecAppendInitializeWorker * diff --git a/src/gausskernel/runtime/executor/nodeBitmapHeapscan.cpp b/src/gausskernel/runtime/executor/nodeBitmapHeapscan.cpp index 3dd4173fd..222db9037 100755 --- a/src/gausskernel/runtime/executor/nodeBitmapHeapscan.cpp +++ b/src/gausskernel/runtime/executor/nodeBitmapHeapscan.cpp @@ -671,12 +671,14 @@ BitmapHeapScanState* ExecInitBitmapHeapScan(BitmapHeapScan* node, EState* estate Partition partition = NULL; Relation partitiontrel = NULL; - /* construct a dummy table relation with the next table partition for scan */ - partition = (Partition)list_nth(scanstate->ss.partitions, 0); - partitiontrel = partitionGetRelation(currentRelation, partition); - scanstate->ss.ss_currentPartition = partitiontrel; - scanstate->ss.ss_currentScanDesc = - abs_tbl_beginscan_bm(partitiontrel, estate->es_snapshot, 0, NULL, &scanstate->ss); + if (scanstate->ss.partitions != NIL) { + /* construct a dummy table relation with the next table partition for scan */ + partition = (Partition)list_nth(scanstate->ss.partitions, 0); + partitiontrel = partitionGetRelation(currentRelation, partition); + scanstate->ss.ss_currentPartition = partitiontrel; + scanstate->ss.ss_currentScanDesc = + abs_tbl_beginscan_bm(partitiontrel, estate->es_snapshot, 0, NULL, &scanstate->ss); + } } } else { scanstate->ss.ss_currentScanDesc = @@ -775,6 +777,7 @@ static void ExecInitPartitionForBitmapHeapScan(BitmapHeapScanState* scanstate, E if (plan->scan.itrs > 0) { LOCKMODE lock = NoLock; Partition tablepartition = NULL; + Oid tablepartitionid = InvalidOid; bool relistarget = false; ListCell* cell = NULL; List* part_seqs = plan->scan.pruningInfo->ls_rangeSelectedPartitions; @@ -784,8 +787,18 @@ static void ExecInitPartitionForBitmapHeapScan(BitmapHeapScanState* scanstate, E lock = (relistarget ? RowExclusiveLock : AccessShareLock); scanstate->ss.lockMode = lock; + if (plan->scan.pruningInfo->paramArg != NULL) { + Param *paramArg = plan->scan.pruningInfo->paramArg; + tablepartitionid = getPartitionOidByParam(currentRelation, paramArg, + &(estate->es_param_list_info->params[paramArg->paramid - 1])); + if (OidIsValid(tablepartitionid)) { + tablepartition = partitionOpen(currentRelation, tablepartitionid, lock); + scanstate->ss.partitions = lappend(scanstate->ss.partitions, tablepartition); + } + return; + } + foreach (cell, part_seqs) { - Oid tablepartitionid = InvalidOid; int partSeq = lfirst_int(cell); /* add table partition to list */ tablepartitionid = getPartitionOidFromSequence(currentRelation, partSeq); diff --git a/src/gausskernel/runtime/executor/nodeBitmapIndexscan.cpp b/src/gausskernel/runtime/executor/nodeBitmapIndexscan.cpp index 0fb3a3407..441828648 100755 --- a/src/gausskernel/runtime/executor/nodeBitmapIndexscan.cpp +++ b/src/gausskernel/runtime/executor/nodeBitmapIndexscan.cpp @@ -372,16 +372,19 @@ BitmapIndexScanState* ExecInitBitmapIndexScan(BitmapIndexScan* node, EState* est /* Initialize table partition and index partition */ ExecInitPartitionForBitmapIndexScan(indexstate, estate, currentrel); - /* get the first index partition */ - currentindex = (Partition)list_nth(indexstate->biss_IndexPartitionList, 0); - indexstate->biss_CurrentIndexPartition = partitionGetRelation(indexstate->biss_RelationDesc, currentindex); + if (indexstate->biss_IndexPartitionList != NIL) { + /* get the first index partition */ + currentindex = (Partition)list_nth(indexstate->biss_IndexPartitionList, 0); + indexstate->biss_CurrentIndexPartition = partitionGetRelation(indexstate->biss_RelationDesc, + currentindex); - ExecCloseScanRelation(currentrel); + ExecCloseScanRelation(currentrel); - indexstate->biss_ScanDesc = abs_idx_beginscan_bitmap(indexstate->biss_CurrentIndexPartition, - estate->es_snapshot, - indexstate->biss_NumScanKeys, - (ScanState*)indexstate); + indexstate->biss_ScanDesc = abs_idx_beginscan_bitmap(indexstate->biss_CurrentIndexPartition, + estate->es_snapshot, + indexstate->biss_NumScanKeys, + (ScanState*)indexstate); + } } } else { /* @@ -465,6 +468,10 @@ static void ExecInitNextPartitionForBitmapIndexScan(BitmapIndexScanState* node) void ExecInitPartitionForBitmapIndexScan(BitmapIndexScanState* indexstate, EState* estate, Relation rel) { BitmapIndexScan* plan = NULL; + Partition tablePartition = NULL; + List* partitionIndexOidList = NIL; + Oid tablepartitionid = InvalidOid; + Oid indexpartitionid = InvalidOid; plan = (BitmapIndexScan*)indexstate->ss.ps.plan; @@ -479,17 +486,48 @@ void ExecInitPartitionForBitmapIndexScan(BitmapIndexScanState* indexstate, EStat ListCell* cell = NULL; List* part_seqs = plan->scan.pruningInfo->ls_rangeSelectedPartitions; - Assert(plan->scan.itrs == plan->scan.pruningInfo->ls_rangeSelectedPartitions->length); relistarget = ExecRelationIsTargetRelation(estate, plan->scan.scanrelid); lock = (relistarget ? RowExclusiveLock : AccessShareLock); indexstate->lockMode = lock; + if (plan->scan.pruningInfo->paramArg != NULL) { + Param *paramArg = plan->scan.pruningInfo->paramArg; + tablepartitionid = getPartitionOidByParam(rel, paramArg, + &(estate->es_param_list_info->params[paramArg->paramid - 1])); + if (OidIsValid(tablepartitionid)) { + tablePartition = partitionOpen(rel, tablepartitionid, lock); + + partitionIndexOidList = PartitionGetPartIndexList(tablePartition); + Assert(PointerIsValid(partitionIndexOidList)); + if (!PointerIsValid(partitionIndexOidList)) { + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmodule(MOD_EXECUTOR), + errmsg("no local indexes found for partition %s BitmapIndexScan", + PartitionGetPartitionName(tablePartition)))); + } + indexpartitionid = searchPartitionIndexOid(indexid, partitionIndexOidList); + list_free_ext(partitionIndexOidList); + partitionClose(rel, tablePartition, NoLock); + + indexpartition = partitionOpen(indexstate->biss_RelationDesc, indexpartitionid, lock); + if (indexpartition->pd_part->indisusable == false) { + ereport(ERROR, + (errcode(ERRCODE_INDEX_CORRUPTED), + errmodule(MOD_EXECUTOR), + errmsg("can't initialize bitmap index scans using unusable local index \"%s\" for partition", + PartitionGetPartitionName(indexpartition)))); + } + /* add index partition to list for the following scan */ + indexstate->biss_IndexPartitionList = lappend(indexstate->biss_IndexPartitionList, indexpartition); + } + return; + } + + Assert(plan->scan.itrs == plan->scan.pruningInfo->ls_rangeSelectedPartitions->length); + foreach (cell, part_seqs) { - Oid tablepartitionid = InvalidOid; int partSeq = lfirst_int(cell); - Oid indexpartitionid = InvalidOid; - Partition tablePartition = NULL; - List* partitionIndexOidList = NIL; /* get index partition list for the special index */ tablepartitionid = getPartitionOidFromSequence(rel, partSeq); diff --git a/src/gausskernel/runtime/executor/nodeGather.cpp b/src/gausskernel/runtime/executor/nodeGather.cpp index 728c5d9fe..0074c3165 100644 --- a/src/gausskernel/runtime/executor/nodeGather.cpp +++ b/src/gausskernel/runtime/executor/nodeGather.cpp @@ -152,6 +152,8 @@ TupleTableSlot *ExecGather(GatherState *node) /* Initialize the workers required to execute Gather node. */ if (!node->pei) { node->pei = ExecInitParallelPlan(node->ps.lefttree, estate, gather->num_workers, node->tuples_needed); + } else { + ExecParallelReinitialize(node->ps.lefttree, node->pei); } /* * Register backend workers. We might not get as many as we @@ -249,12 +251,13 @@ TupleTableSlot *ExecGather(GatherState *node) * frees any storage allocated through C routines. * ---------------------------------------------------------------- */ -void ExecEndGather(GatherState *node) +void ExecEndGather(GatherState* node) { + /* let children clean up first */ + ExecEndNode(outerPlanState(node)); ExecShutdownGather(node); ExecFreeExprContext(&node->ps); (void)ExecClearTuple(node->ps.ps_ResultTupleSlot); - ExecEndNode(outerPlanState(node)); } /* @@ -412,21 +415,39 @@ void ExecShutdownGather(GatherState *node) * Re-initialize the workers and rescans a relation via them. * ---------------------------------------------------------------- */ -void ExecReScanGather(GatherState *node) +void ExecReScanGather(GatherState* node) { - /* - * Re-initialize the parallel workers to perform rescan of relation. - * We want to gracefully shutdown all the workers so that they - * should be able to propagate any error or other information to master - * backend before dying. Parallel context will be reused for rescan. - */ + Gather* gather = (Gather*)node->ps.plan; + PlanState* outerPlan = outerPlanState(node); + + /* Make sure any existing workers are gracefully shut down */ ExecShutdownGatherWorkers(node); + /* Mark node so that shared state will be rebuilt at next call */ node->initialized = false; - if (node->pei) - ExecParallelReinitialize(node->pei); + /* + * Set child node's chgParam to tell it that the next scan might deliver a + * different set of rows within the leader process. (The overall rowset + * shouldn't change, but the leader process's subset might; hence nodes + * between here and the parallel table scan node mustn't optimize on the + * assumption of an unchanging rowset.) + */ + if (gather->rescan_param >= 0) { + outerPlan->chgParam = bms_add_member(outerPlan->chgParam, gather->rescan_param); + } - ExecReScan(node->ps.lefttree); + /* + * If chgParam of subnode is not null then plan will be re-scanned by + * first ExecProcNode. Note: because this does nothing if we have a + * rescan_param, it's currently guaranteed that parallel-aware child nodes + * will not see a ReScan call until after they get a ReInitializeDSM call. + * That ordering might not be something to rely on, though. A good rule + * of thumb is that ReInitializeDSM should reset only shared state, ReScan + * should reset only local state, and anything that depends on both of + * those steps being finished must wait until the first ExecProcNode call. + */ + if (outerPlan->chgParam == NULL) { + ExecReScan(outerPlan); + } } - diff --git a/src/gausskernel/runtime/executor/nodeHash.cpp b/src/gausskernel/runtime/executor/nodeHash.cpp index e37707375..48455de24 100755 --- a/src/gausskernel/runtime/executor/nodeHash.cpp +++ b/src/gausskernel/runtime/executor/nodeHash.cpp @@ -24,6 +24,7 @@ #include #include #include "access/hash.h" +#include "access/parallel.h" #include "catalog/pg_partition_fn.h" #include "catalog/pg_statistic.h" #include "commands/tablespace.h" @@ -32,11 +33,13 @@ #include "executor/nodeHash.h" #include "executor/nodeHashjoin.h" #include "miscadmin.h" +#include "pgstat.h" #include "optimizer/clauses.h" #include "optimizer/streamplan.h" #include "pgstat.h" #include "pgxc/pgxc.h" #include "utils/anls_opt.h" +#include "utils/atomic.h" #include "utils/dynahash.h" #include "utils/lsyscache.h" #include "utils/memprot.h" @@ -49,12 +52,29 @@ #include "workload/workload.h" static void ExecHashIncreaseNumBatches(HashJoinTable hashtable); +static void ExecParallelHashIncreaseNumBatches(HashJoinTable hashtable); +static void ExecParallelHashIncreaseNumBuckets(HashJoinTable hashtable); static void ExecHashBuildSkewHash(HashJoinTable hashtable, Hash* node, int mcvsToUse); static void ExecHashSkewTableInsert(HashJoinTable hashtable, TupleTableSlot* slot, uint32 hashvalue, int bucketNumber); static void ExecHashRemoveNextSkewBucket(HashJoinTable hashtable); static void ExecHashIncreaseBuckets(HashJoinTable hashtable); static void* dense_alloc(HashJoinTable hashtable, Size size); +static HashJoinTuple ExecParallelHashTupleAlloc(HashJoinTable hashtable, size_t size, HashJoinTuple* shared); +static void MultiExecPrivateHash(HashState* node); +static void MultiExecParallelHash(HashState* node); +static inline HashJoinTuple ExecParallelHashFirstTuple(HashJoinTable table, int bucketno); +static inline HashJoinTuple ExecParallelHashNextTuple(HashJoinTable table, HashJoinTuple tuple); +static inline void ExecParallelHashPushTuple(HashJoinTuple* head, HashJoinTuple tuple, HashJoinTuple tuple_shared); +static void ExecParallelHashJoinSetUpBatches(HashJoinTable hashtable, int nbatch); +static void ExecParallelHashEnsureBatchAccessors(HashJoinTable hashtable); +static void ExecParallelHashRepartitionFirst(HashJoinTable hashtable); +static void ExecParallelHashRepartitionRest(HashJoinTable hashtable); +static HashMemoryChunk ExecParallelHashPopChunkQueue(HashJoinTable table, HashMemoryChunk* shared); +static bool ExecParallelHashTuplePrealloc(HashJoinTable hashtable, int batchno, size_t size); +static void ExecParallelHashMergeCounters(HashJoinTable hashtable); +static void ExecParallelHashCloseBatchAccessors(HashJoinTable hashtable); + /* ---------------------------------------------------------------- * ExecHash * @@ -79,13 +99,6 @@ TupleTableSlot* ExecHash(void) */ Node* MultiExecHash(HashState* node) { - PlanState* outerNode = NULL; - List* hashkeys = NIL; - HashJoinTable hashtable; - TupleTableSlot* slot = NULL; - ExprContext* econtext = NULL; - uint32 hashvalue; - /* must provide our own instrumentation support */ if (node->ps.instrument) { InstrStartNode(node->ps.instrument); @@ -94,6 +107,43 @@ Node* MultiExecHash(HashState* node) node->hashtable->spill_size = &node->spill_size; } + if (node->parallel_state != NULL) { + MultiExecParallelHash(node); + } else { + MultiExecPrivateHash(node); + } + + /* must provide our own instrumentation support */ + if (node->ps.instrument) { + InstrStopNode(node->ps.instrument, node->hashtable->partialTuples); + } + + /* + * We do not return the hash table directly because it's not a subtype of + * Node, and so would violate the MultiExecProcNode API. Instead, our + * parent Hashjoin node is expected to know how to fish it out of our node + * state. Ugly but not really worth cleaning up, since Hashjoin knows + * quite a bit more about Hash besides that. + */ + return NULL; +} + +/* ---------------------------------------------------------------- + * MultiExecPrivateHash + * + * parallel-oblivious version, building a backend-private + * hash table and (if necessary) batch files. + * ---------------------------------------------------------------- + */ +static void MultiExecPrivateHash(HashState* node) +{ + PlanState* outerNode = NULL; + List* hashkeys = NIL; + HashJoinTable hashtable = NULL; + TupleTableSlot* slot = NULL; + ExprContext* econtext = NULL; + uint32 hashvalue; + /* * get state info from node */ @@ -136,27 +186,13 @@ Node* MultiExecHash(HashState* node) } } (void)pgstat_report_waitstatus(oldStatus); - + /* analysis hash table information created in memory */ - if (anls_opt_is_on(ANLS_HASH_CONFLICT)) + if (anls_opt_is_on(ANLS_HASH_CONFLICT)) { ExecHashTableStats(hashtable, node->ps.plan->plan_node_id); - - /* must provide our own instrumentation support */ - if (node->ps.instrument) { - InstrStopNode(node->ps.instrument, hashtable->totalTuples); - node->ps.instrument->sorthashinfo.nbatch = hashtable->nbatch; - node->ps.instrument->sorthashinfo.nbuckets = hashtable->nbuckets; - node->ps.instrument->sorthashinfo.nbatch_original = hashtable->nbatch_original; - node->ps.instrument->sorthashinfo.spacePeak = hashtable->spacePeak; - if (hashtable->width[0] > 0) { - hashtable->width[1] = hashtable->width[1] / hashtable->width[0]; - hashtable->width[0] = -1; - } - node->ps.instrument->width = (int)hashtable->width[1]; - node->ps.instrument->sysBusy = hashtable->causedBySysRes; - node->ps.instrument->spreadNum = hashtable->spreadNum; } - + + hashtable->partialTuples = hashtable->totalTuples; /* * We do not return the hash table directly because it's not a subtype of * Node, and so would violate the MultiExecProcNode API. Instead, our @@ -164,7 +200,6 @@ Node* MultiExecHash(HashState* node) * state. Ugly but not really worth cleaning up, since Hashjoin knows * quite a bit more about Hash besides that. */ - return NULL; } /* ---------------------------------------------------------------- @@ -222,6 +257,143 @@ HashState* ExecInitHash(Hash* node, EState* estate, int eflags) return hashstate; } +/* ---------------------------------------------------------------- + * MultiExecParallelHash + * + * parallel-aware version, building a shared hash table and + * (if necessary) batch files using the combined effort of + * a set of co-operating backends. + * ---------------------------------------------------------------- + */ +static void MultiExecParallelHash(HashState* node) +{ + ParallelHashJoinState* pstate = NULL; + PlanState* outerNode = NULL; + List* hashkeys = NIL; + HashJoinTable hashtable = NULL; + TupleTableSlot* slot = NULL; + ExprContext* econtext = NULL; + uint32 hashvalue; + Barrier* build_barrier = NULL; + int i; + + /* + * get state info from node + */ + outerNode = outerPlanState(node); + hashtable = node->hashtable; + + /* + * set expression context + */ + hashkeys = node->hashkeys; + econtext = node->ps.ps_ExprContext; + + /* + * Synchronize the parallel hash table build. At this stage we know that + * the shared hash table has been or is being set up by + * ExecHashTableCreate(), but we don't know if our peers have returned + * from there or are here in MultiExecParallelHash(), and if so how far + * through they are. To find out, we check the build_barrier phase then + * and jump to the right step in the build algorithm. + */ + pstate = hashtable->parallel_state; + build_barrier = &pstate->build_barrier; + Assert(BarrierPhase(build_barrier) >= PHJ_BUILD_ALLOCATING); + switch (BarrierPhase(build_barrier)) { + case PHJ_BUILD_ALLOCATING: + + /* + * Either I just allocated the initial hash table in + * ExecHashTableCreate(), or someone else is doing that. Either + * way, wait for everyone to arrive here so we can proceed. + */ + (void)BarrierArriveAndWait(build_barrier, WAIT_EVENT_HASH_BUILD_ALLOCATING); + /* Fall through. */ + + case PHJ_BUILD_HASHING_INNER: + + /* + * It's time to begin hashing, or if we just arrived here then + * hashing is already underway, so join in that effort. While + * hashing we have to be prepared to help increase the number of + * batches or buckets at any time, and if we arrived here when + * that was already underway we'll have to help complete that work + * immediately so that it's safe to access batches and buckets + * below. + */ + if (PHJ_GROW_BATCHES_PHASE(BarrierAttach(&pstate->grow_batches_barrier)) != PHJ_GROW_BATCHES_ELECTING) { + ExecParallelHashIncreaseNumBatches(hashtable); + } + if (PHJ_GROW_BUCKETS_PHASE(BarrierAttach(&pstate->grow_buckets_barrier)) != PHJ_GROW_BUCKETS_ELECTING) { + ExecParallelHashIncreaseNumBuckets(hashtable); + } + ExecParallelHashEnsureBatchAccessors(hashtable); + ExecParallelHashTableSetCurrentBatch(hashtable, 0); + for (;;) { + slot = ExecProcNode(outerNode); + if (TupIsNull(slot)) { + break; + } + econtext->ecxt_innertuple = slot; + if (ExecHashGetHashValue(hashtable, econtext, hashkeys, false, hashtable->keepNulls, &hashvalue)) { + ExecParallelHashTableInsert(hashtable, slot, hashvalue); + } + hashtable->partialTuples++; + } + /* + * Make sure that any tuples we wrote to disk are visible to + * others before anyone tries to load them. + */ + for (i = 0; i < hashtable->nbatch; ++i) { + sts_end_write(hashtable->batches[i].inner_tuples); + } + + /* + * Update shared counters. We need an accurate total tuple count + * to control the empty table optimization. + */ + ExecParallelHashMergeCounters(hashtable); + + (void)BarrierDetach(&pstate->grow_buckets_barrier); + (void)BarrierDetach(&pstate->grow_batches_barrier); + /* + * Wait for everyone to finish building and flushing files and + * counters. + */ + if (BarrierArriveAndWait(build_barrier, WAIT_EVENT_HASH_BUILD_HASHING_INNER)) { + /* + * Elect one backend to disable any further growth. Batches + * are now fixed. While building them we made sure they'd fit + * in our memory budget when we load them back in later (or we + * tried to do that and gave up because we detected extreme + * skew). + */ + pstate->growth = PHJ_GROWTH_DISABLED; + } + break; + default: + break; + } + + /* + * We're not yet attached to a batch. We all agree on the dimensions and + * number of inner tuples (for the empty table optimization). + */ + hashtable->curbatch = -1; + hashtable->nbuckets = pstate->nbuckets; + hashtable->log2_nbuckets = my_log2(hashtable->nbuckets); + hashtable->totalTuples = pstate->total_tuples; + ExecParallelHashEnsureBatchAccessors(hashtable); + + /* + * The next synchronization point is in ExecHashJoin's HJ_BUILD_HASHTABLE + * case, which will bring the build phase to PHJ_BUILD_DONE (if it isn't + * there already). + */ + Assert(BarrierPhase(build_barrier) == PHJ_BUILD_HASHING_OUTER || BarrierPhase(build_barrier) == PHJ_BUILD_DONE); +} + /* --------------------------------------------------------------- * ExecEndHash * @@ -250,12 +422,15 @@ void ExecEndHash(HashState* node) * create an empty hashtable data structure for hashjoin. * ---------------------------------------------------------------- */ -HashJoinTable ExecHashTableCreate(Hash* node, List* hashOperators, bool keepNulls) +HashJoinTable ExecHashTableCreate(HashState* state, List* hashOperators, bool keepNulls) { - HashJoinTable hashtable; + Hash* node = (Hash*)state->ps.plan; + HashJoinTable hashtable = NULL; Plan* outerNode = NULL; + size_t space_allowed; int nbuckets; int nbatch; + double rows; int num_skew_mcvs; int log2_nbuckets; int nkeys; @@ -270,11 +445,21 @@ HashJoinTable ExecHashTableCreate(Hash* node, List* hashOperators, bool keepNull * "outer" subtree of this node, but the inner relation of the hashjoin). * Compute the appropriate size of the hash table. */ + outerNode = outerPlan(node); - ExecChooseHashTableSize(PLAN_LOCAL_ROWS(outerNode) / SET_DOP(node->plan.dop), + /* + * If this is shared hash table with a partial plan, then we can't use + * outerNode->plan_rows to estimate its size. We need an estimate of the + * total number of rows across all copies of the partial plan. + */ + rows = node->plan.parallel_aware ? node->rows_total : (PLAN_LOCAL_ROWS(outerNode) / SET_DOP(node->plan.dop)); + ExecChooseHashTableSize(rows, outerNode->plan_width, OidIsValid(node->skewTable), + state->parallel_state != NULL, + (state->parallel_state != NULL) ? (state->parallel_state->nparticipants - 1) : 0, + &space_allowed, &nbuckets, &nbatch, &num_skew_mcvs, @@ -308,7 +493,7 @@ HashJoinTable ExecHashTableCreate(Hash* node, List* hashOperators, bool keepNull hashtable = (HashJoinTable)palloc(sizeof(HashJoinTableData)); hashtable->nbuckets = nbuckets; hashtable->log2_nbuckets = log2_nbuckets; - hashtable->buckets = NULL; + hashtable->buckets.unshared = NULL; hashtable->keepNulls = keepNulls; hashtable->skewEnabled = false; hashtable->skewBucket = NULL; @@ -321,15 +506,21 @@ HashJoinTable ExecHashTableCreate(Hash* node, List* hashOperators, bool keepNull hashtable->nbatch_outstart = nbatch; hashtable->growEnabled = true; hashtable->totalTuples = 0; + hashtable->partialTuples = 0; hashtable->innerBatchFile = NULL; hashtable->outerBatchFile = NULL; hashtable->spaceUsed = 0; hashtable->spacePeak = 0; - hashtable->spaceAllowed = local_work_mem * 1024L; + hashtable->spaceAllowed = (int64)space_allowed; hashtable->spaceUsedSkew = 0; hashtable->spaceAllowedSkew = hashtable->spaceAllowed * SKEW_WORK_MEM_PERCENT / 100; hashtable->chunks = NULL; + hashtable->current_chunk = NULL; + hashtable->parallel_state = state->parallel_state; + hashtable->area = t_thrd.bgworker_cxt.memCxt; + hashtable->batches = NULL; + hashtable->width[0] = hashtable->width[1] = 0; hashtable->causedBySysRes = false; @@ -385,9 +576,10 @@ HashJoinTable ExecHashTableCreate(Hash* node, List* hashOperators, bool keepNull /* Allocate data that will live for the life of the hashjoin */ oldcxt = MemoryContextSwitchTo(hashtable->hashCxt); - if (nbatch > 1) { + if (nbatch > 1 && hashtable->parallel_state == NULL) { /* - * allocate and initialize the file arrays in hashCxt + * allocate and initialize the file arrays in hashCxt (not needed for + * parallel case which uses shared tuplestores instead of raw files) */ hashtable->innerBatchFile = (BufFile**)palloc0(nbatch * sizeof(BufFile*)); hashtable->outerBatchFile = (BufFile**)palloc0(nbatch * sizeof(BufFile*)); @@ -395,23 +587,72 @@ HashJoinTable ExecHashTableCreate(Hash* node, List* hashOperators, bool keepNull /* ... but make sure we have temp tablespaces established for them */ PrepareTempTablespaces(); } + (void)MemoryContextSwitchTo(oldcxt); + if (hashtable->parallel_state) { + ParallelHashJoinState* pstate = hashtable->parallel_state; + Barrier* build_barrier = NULL; - /* - * Prepare context for the first-scan space allocations; allocate the - * hashbucket array therein, and set each bucket "empty". - */ - MemoryContextSwitchTo(hashtable->batchCxt); + /* + * Attach to the build barrier. The corresponding detach operation is + * in ExecHashTableDetach. Note that we won't attach to the + * batch_barrier for batch 0 yet. We'll attach later and start it out + * in PHJ_BATCH_PROBING phase, because batch 0 is allocated up front + * and then loaded while hashing (the standard hybrid hash join + * algorithm), and we'll coordinate that using build_barrier. + */ + build_barrier = &pstate->build_barrier; + BarrierAttach(build_barrier); - hashtable->buckets = (HashJoinTuple*)palloc0(nbuckets * sizeof(HashJoinTuple)); + /* + * So far we have no idea whether there are any other participants, + * and if so, what phase they are working on. The only thing we care + * about at this point is whether someone has already created the + * SharedHashJoinBatch objects and the hash table for batch 0. One + * backend will be elected to do that now if necessary. + */ + if (BarrierPhase(build_barrier) == PHJ_BUILD_ELECTING && + BarrierArriveAndWait(build_barrier, WAIT_EVENT_HASH_BUILD_ELECTING)) { + pstate->nbatch = nbatch; + pstate->space_allowed = space_allowed; + pstate->growth = PHJ_GROWTH_OK; - /* - * Set up for skew optimization, if possible and there's a need for more - * than one batch. (In a one-batch join, there's no point in it.) - */ - if (nbatch > 1) - ExecHashBuildSkewHash(hashtable, node, num_skew_mcvs); + /* Set up the shared state for coordinating batches. */ + ExecParallelHashJoinSetUpBatches(hashtable, nbatch); - MemoryContextSwitchTo(oldcxt); + /* + * Allocate batch 0's hash table up front so we can load it + * directly while hashing. + */ + pstate->nbuckets = nbuckets; + ExecParallelHashTableAlloc(hashtable, 0); + } + + /* + * The next Parallel Hash synchronization point is in + * MultiExecParallelHash(), which will progress it all the way to + * PHJ_BUILD_DONE. The caller must not return control from this + * executor node between now and then. + */ + } else { + /* + * Prepare context for the first-scan space allocations; allocate the + * hashbucket array therein, and set each bucket "empty". + */ + (void)MemoryContextSwitchTo(hashtable->batchCxt); + + hashtable->buckets.unshared = (HashJoinTuple*)palloc0(nbuckets * sizeof(HashJoinTuple)); + + /* + * Set up for skew optimization, if possible and there's a need for + * more than one batch. (In a one-batch join, there's no point in + * it.) + */ + if (nbatch > 1) { + ExecHashBuildSkewHash(hashtable, node, num_skew_mcvs); + } + + (void)MemoryContextSwitchTo(oldcxt); + } return hashtable; } @@ -493,8 +734,9 @@ double ExecChooseHashTableMaxTuples(int tupwidth, bool useskew, bool vectorized, * * This is exported so that the planner's costsize.c can use it. */ -void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, int* numbuckets, int* numbatches, - int* num_skew_mcvs, int4 localWorkMem, bool vectorized, OpMemInfo* memInfo) +void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, bool try_combined_work_mem, + int parallel_workers, size_t* space_allowed, int* numbuckets, int* numbatches, int* num_skew_mcvs, + int4 localWorkMem, bool vectorized, OpMemInfo* memInfo) { int tupsize; double inner_rel_bytes; @@ -538,6 +780,16 @@ void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, int* nu */ hash_table_bytes = localWorkMem * 1024L; + /* + * Parallel Hash tries to use the combined work_mem of all workers to + * avoid the need to batch. If that won't work, it falls back to work_mem + * per worker and tries to process batches in parallel. + */ + if (try_combined_work_mem) { + hash_table_bytes += hash_table_bytes * parallel_workers; + } + + *space_allowed = (size_t)hash_table_bytes; /* * If skew optimization is possible, estimate the number of skew buckets * that will fit in the memory allowed, and decrement the assumed space @@ -581,7 +833,7 @@ void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, int* nu * Note that both nbuckets and nbatch must be powers of 2 to make * ExecHashGetBucketAndBatch fast. */ - max_pointers = (localWorkMem * 1024L) / hash_header_size; + max_pointers = *space_allowed / (int64)hash_header_size; max_pointers = Min(max_pointers, (long)(MaxAllocSize / hash_header_size)); /* If max_pointers isn't a power of 2, must round it down to one */ mppow2 = 1UL << my_log2(max_pointers); @@ -619,6 +871,26 @@ void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, int* nu double max_batch; int64 bucket_size; + /* + * If Parallel Hash with combined work_mem would still need multiple + * batches, we'll have to fall back to regular work_mem budget. + */ + if (try_combined_work_mem) { + ExecChooseHashTableSize(ntuples, + tupwidth, + useskew, + false, + parallel_workers, + space_allowed, + numbuckets, + numbatches, + num_skew_mcvs, + localWorkMem, + vectorized, + memInfo); + return; + } + /* * Estimate the number of buckets we'll want to have when work_mem is * entirely full. Each bucket will contain a bucket pointer plus @@ -744,13 +1016,17 @@ void ExecHashTableDestroy(HashJoinTable hashtable) /* * Make sure all the temp files are closed. We skip batch 0, since it * can't have any temp files (and the arrays might not even exist if - * nbatch is only 1). + * nbatch is only 1). Parallel hash joins don't use these files. */ - for (i = 1; i < hashtable->nbatch; i++) { - if (hashtable->innerBatchFile[i]) - BufFileClose(hashtable->innerBatchFile[i]); - if (hashtable->outerBatchFile[i]) - BufFileClose(hashtable->outerBatchFile[i]); + if (hashtable->innerBatchFile != NULL) { + for (i = 1; i < hashtable->nbatch; i++) { + if (hashtable->innerBatchFile[i]) { + BufFileClose(hashtable->innerBatchFile[i]); + } + if (hashtable->outerBatchFile[i]) { + BufFileClose(hashtable->outerBatchFile[i]); + } + } } /* Free the unused buffers */ @@ -820,7 +1096,7 @@ static void ExecHashIncreaseNumBatches(HashJoinTable hashtable) securec_check(rc, "\0", "\0"); } - MemoryContextSwitchTo(oldcxt); + (void)MemoryContextSwitchTo(oldcxt); hashtable->nbatch = nbatch; @@ -829,13 +1105,12 @@ static void ExecHashIncreaseNumBatches(HashJoinTable hashtable) * no longer of the current batch. */ ninmemory = nfreed = 0; - /* * We will scan through the chunks directly, so that we can reset the * buckets now and not have to keep track which tuples in the buckets have * already been processed. We will free the old chunks as we go. */ - rc = memset_s(hashtable->buckets, + rc = memset_s(hashtable->buckets.unshared, sizeof(HashJoinTuple*) * hashtable->nbuckets, 0, sizeof(HashJoinTuple*) * hashtable->nbuckets); @@ -845,7 +1120,7 @@ static void ExecHashIncreaseNumBatches(HashJoinTable hashtable) /* so, let's scan through the old chunks, and all tuples in each chunk */ while (oldchunks != NULL) { - HashMemoryChunk nextchunk = oldchunks->next; + HashMemoryChunk nextchunk = oldchunks->next.unshared; /* position within the buffer (up to oldchunks->used) */ size_t idx = 0; @@ -867,8 +1142,8 @@ static void ExecHashIncreaseNumBatches(HashJoinTable hashtable) securec_check(rc, "\0", "\0"); /* and add it back to the appropriate bucket */ - copyTuple->next = hashtable->buckets[bucketno]; - hashtable->buckets[bucketno] = copyTuple; + copyTuple->next.unshared = hashtable->buckets.unshared[bucketno]; + hashtable->buckets.unshared[bucketno] = copyTuple; } else { /* dump it out */ Assert(batchno > curbatch); @@ -911,6 +1186,415 @@ static void ExecHashIncreaseNumBatches(HashJoinTable hashtable) } } +/* + * ExecParallelHashIncreaseNumBatches + * Every participant attached to grow_barrier must run this function + * when it observes growth == PHJ_GROWTH_NEED_MORE_BATCHES. + */ +static void ExecParallelHashIncreaseNumBatches(HashJoinTable hashtable) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + int i; + + Assert(BarrierPhase(&pstate->build_barrier) == PHJ_BUILD_HASHING_INNER); + + /* + * It's unlikely, but we need to be prepared for new participants to show + * up while we're in the middle of this operation so we need to switch on + * barrier phase here. + */ + switch (PHJ_GROW_BATCHES_PHASE(BarrierPhase(&pstate->grow_batches_barrier))) { + case PHJ_GROW_BATCHES_ELECTING: + + /* + * Elect one participant to prepare to grow the number of batches. + * This involves reallocating or resetting the buckets of batch 0 + * in preparation for all participants to begin repartitioning the + * tuples. + */ + if (BarrierArriveAndWait(&pstate->grow_batches_barrier, WAIT_EVENT_HASH_GROW_BATCHES_ELECTING)) { + dsa_pointer_atomic* buckets = NULL; + ParallelHashJoinBatch* old_batch0 = NULL; + int new_nbatch; + int i; + + /* Move the old batch out of the way. */ + old_batch0 = hashtable->batches[0].shared; + pstate->old_batches = pstate->batches; + pstate->old_nbatch = hashtable->nbatch; + pstate->batches = InvalidDsaPointer; + + /* Free this backend's old accessors. */ + ExecParallelHashCloseBatchAccessors(hashtable); + + /* Figure out how many batches to use. */ + if (hashtable->nbatch == 1) { + /* + * We are going from single-batch to multi-batch. We need + * to switch from one large combined memory budget to the + * regular work_mem budget. + */ + pstate->space_allowed = ((size_t)u_sess->attr.attr_memory.work_mem) * 1024L; + + /* + * The combined work_mem of all participants wasn't + * enough. Therefore one batch per participant would be + * approximately equivalent and would probably also be + * insufficient. So try two batches per particiant, + * rounded up to a power of two. + */ + new_nbatch = 1 << my_log2(pstate->nparticipants * 2); + } else { + /* + * We were already multi-batched. Try doubling the number + * of batches. + */ + new_nbatch = hashtable->nbatch * 2; + } + + /* Allocate new larger generation of batches. */ + Assert(hashtable->nbatch == pstate->nbatch); + ExecParallelHashJoinSetUpBatches(hashtable, new_nbatch); + Assert(hashtable->nbatch == pstate->nbatch); + + /* Replace or recycle batch 0's bucket array. */ + if (pstate->old_nbatch == 1) { + double dtuples; + double dbuckets; + int new_nbuckets; + + /* + * We probably also need a smaller bucket array. How many + * tuples do we expect per batch, assuming we have only + * half of them so far? Normally we don't need to change + * the bucket array's size, because the size of each batch + * stays the same as we add more batches, but in this + * special case we move from a large batch to many smaller + * batches and it would be wasteful to keep the large + * array. + */ + dtuples = (old_batch0->ntuples * 2.0) / new_nbatch; + dbuckets = ceil(dtuples / NTUP_PER_BUCKET); + dbuckets = Min(dbuckets, MaxAllocSize / sizeof(dsa_pointer_atomic)); + new_nbuckets = (int)dbuckets; + new_nbuckets = Max(new_nbuckets, MIN_HASH_BUCKET_SIZE); + new_nbuckets = 1 << my_log2(new_nbuckets); + dsa_free(hashtable->area, old_batch0->buckets); + hashtable->batches[0].shared->buckets = + MemoryContextAlloc(hashtable->area, sizeof(dsa_pointer_atomic) * new_nbuckets); + buckets = (dsa_pointer_atomic*)hashtable->batches[0].shared->buckets; + for (i = 0; i < new_nbuckets; ++i) { + dsa_pointer_atomic_init(&buckets[i], 0); + } + pstate->nbuckets = new_nbuckets; + } else { + /* Recycle the existing bucket array. */ + hashtable->batches[0].shared->buckets = old_batch0->buckets; + buckets = (dsa_pointer_atomic*)dsa_get_address(hashtable->area, old_batch0->buckets); + for (i = 0; i < hashtable->nbuckets; ++i) { + dsa_pointer_atomic_write(&buckets[i], 0); + } + } + + /* Move all chunks to the work queue for parallel processing. */ + pstate->chunk_work_queue = old_batch0->chunks; + + /* Disable further growth temporarily while we're growing. */ + pstate->growth = PHJ_GROWTH_DISABLED; + } else { + /* All other participants just flush their tuples to disk. */ + ExecParallelHashCloseBatchAccessors(hashtable); + } + /* Fall through. */ + + case PHJ_GROW_BATCHES_ALLOCATING: + /* Wait for the above to be finished. */ + (void)BarrierArriveAndWait(&pstate->grow_batches_barrier, WAIT_EVENT_HASH_GROW_BATCHES_ALLOCATING); + /* Fall through. */ + + case PHJ_GROW_BATCHES_REPARTITIONING: + /* Make sure that we have the current dimensions and buckets. */ + ExecParallelHashEnsureBatchAccessors(hashtable); + ExecParallelHashTableSetCurrentBatch(hashtable, 0); + /* Then partition, flush counters. */ + ExecParallelHashRepartitionFirst(hashtable); + ExecParallelHashRepartitionRest(hashtable); + ExecParallelHashMergeCounters(hashtable); + /* Wait for the above to be finished. */ + (void)BarrierArriveAndWait(&pstate->grow_batches_barrier, WAIT_EVENT_HASH_GROW_BATCHES_REPARTITIONING); + /* Fall through. */ + + case PHJ_GROW_BATCHES_DECIDING: + + /* + * Elect one participant to clean up and decide whether further + * repartitioning is needed, or should be disabled because it's + * not helping. + */ + if (BarrierArriveAndWait(&pstate->grow_batches_barrier, WAIT_EVENT_HASH_GROW_BATCHES_DECIDING)) { + bool space_exhausted = false; + bool extreme_skew_detected = false; + + /* Make sure that we have the current dimensions and buckets. */ + ExecParallelHashEnsureBatchAccessors(hashtable); + ExecParallelHashTableSetCurrentBatch(hashtable, 0); + + /* Are any of the new generation of batches exhausted? */ + for (i = 0; i < hashtable->nbatch; ++i) { + ParallelHashJoinBatch* batch = hashtable->batches[i].shared; + + if (batch->space_exhausted || batch->estimated_size > pstate->space_allowed) { + int parent; + + space_exhausted = true; + + /* + * Did this batch receive ALL of the tuples from its + * parent batch? That would indicate that further + * repartitioning isn't going to help (the hash values + * are probably all the same). + */ + parent = i % pstate->old_nbatch; + if (batch->ntuples == hashtable->batches[parent].shared->old_ntuples) { + extreme_skew_detected = true; + } + } + } + + /* Don't keep growing if it's not helping or we'd overflow. */ + if (extreme_skew_detected || hashtable->nbatch >= INT_MAX / 2) { + pstate->growth = PHJ_GROWTH_DISABLED; + } else if (space_exhausted) { + pstate->growth = PHJ_GROWTH_NEED_MORE_BATCHES; + } else { + pstate->growth = PHJ_GROWTH_OK; + } + + /* Free the old batches in shared memory. */ + dsa_free(hashtable->area, pstate->old_batches); + pstate->old_batches = InvalidDsaPointer; + } + /* Fall through. */ + + case PHJ_GROW_BATCHES_FINISHING: + /* Wait for the above to complete. */ + (void)BarrierArriveAndWait(&pstate->grow_batches_barrier, WAIT_EVENT_HASH_GROW_BATCHES_FINISHING); + break; + default: + break; + } +} + +/* + * Repartition the tuples currently loaded into memory for inner batch 0 + * because the number of batches has been increased. Some tuples are retained + * in memory and some are written out to a later batch. + */ +static void ExecParallelHashRepartitionFirst(HashJoinTable hashtable) +{ + HashMemoryChunk chunk_shared; + HashMemoryChunk chunk; + + Assert(hashtable->nbatch == hashtable->parallel_state->nbatch); + + while ((chunk = ExecParallelHashPopChunkQueue(hashtable, &chunk_shared))) { + size_t idx = 0; + + /* Repartition all tuples in this chunk. */ + while (idx < chunk->used) { + HashJoinTuple hashTuple = (HashJoinTuple)(chunk->data + idx); + MinimalTuple tuple = HJTUPLE_MINTUPLE(hashTuple); + HashJoinTuple copyTuple; + HashJoinTuple shared; + int bucketno; + int batchno; + + ExecHashGetBucketAndBatch(hashtable, hashTuple->hashvalue, &bucketno, &batchno); + + Assert(batchno < hashtable->nbatch); + if (batchno == 0) { + /* It still belongs in batch 0. Copy to a new chunk. */ + copyTuple = ExecParallelHashTupleAlloc(hashtable, HJTUPLE_OVERHEAD + tuple->t_len, &shared); + copyTuple->hashvalue = hashTuple->hashvalue; + errno_t rc = memcpy_s(HJTUPLE_MINTUPLE(copyTuple), tuple->t_len, tuple, tuple->t_len); + securec_check(rc, "\0", "\0"); + ExecParallelHashPushTuple(&hashtable->buckets.shared[bucketno], copyTuple, shared); + } else { + size_t tuple_size = MAXALIGN(HJTUPLE_OVERHEAD + tuple->t_len); + + /* It belongs in a later batch. */ + hashtable->batches[batchno].estimated_size += tuple_size; + sts_puttuple(hashtable->batches[batchno].inner_tuples, &hashTuple->hashvalue, tuple); + } + + /* Count this tuple. */ + ++hashtable->batches[0].old_ntuples; + ++hashtable->batches[batchno].ntuples; + + idx += MAXALIGN(HJTUPLE_OVERHEAD + HJTUPLE_MINTUPLE(hashTuple)->t_len); + } + + /* Free this chunk. */ + dsa_free(hashtable->area, chunk_shared); + + CHECK_FOR_INTERRUPTS(); + } +} + +/* + * Help repartition inner batches 1..n. + */ +static void ExecParallelHashRepartitionRest(HashJoinTable hashtable) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + int old_nbatch = pstate->old_nbatch; + SharedTuplestoreAccessor** old_inner_tuples = NULL; + ParallelHashJoinBatch* old_batches = NULL; + int i; + + /* Get our hands on the previous generation of batches. */ + old_batches = (ParallelHashJoinBatch*)dsa_get_address(hashtable->area, pstate->old_batches); + old_inner_tuples = (SharedTuplestoreAccessor**)palloc0(sizeof(SharedTuplestoreAccessor*) * old_nbatch); + for (i = 1; i < old_nbatch; ++i) { + ParallelHashJoinBatch* shared = NthParallelHashJoinBatch(old_batches, i); + + old_inner_tuples[i] = sts_attach( + ParallelHashJoinBatchInner(shared), t_thrd.bgworker_cxt.ParallelWorkerNumber + 1, &pstate->fileset); + } + + /* Join in the effort to repartition them. */ + for (i = 1; i < old_nbatch; ++i) { + MinimalTuple tuple = NULL; + uint32 hashvalue; + + /* Scan one partition from the previous generation. */ + sts_begin_parallel_scan(old_inner_tuples[i]); + while ((tuple = sts_parallel_scan_next(old_inner_tuples[i], &hashvalue))) { + size_t tuple_size = MAXALIGN(HJTUPLE_OVERHEAD + tuple->t_len); + int bucketno; + int batchno; + + /* Decide which partition it goes to in the new generation. */ + ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno); + + hashtable->batches[batchno].estimated_size += tuple_size; + ++hashtable->batches[batchno].ntuples; + ++hashtable->batches[i].old_ntuples; + + /* Store the tuple its new batch. */ + sts_puttuple(hashtable->batches[batchno].inner_tuples, &hashvalue, tuple); + + CHECK_FOR_INTERRUPTS(); + } + sts_end_parallel_scan(old_inner_tuples[i]); + pfree(old_inner_tuples[i]); + } + + pfree(old_inner_tuples); +} + +/* + * Transfer the backend-local per-batch counters to the shared totals. + */ +static void ExecParallelHashMergeCounters(HashJoinTable hashtable) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + int i; + + (void)LWLockAcquire(&pstate->lock, LW_EXCLUSIVE); + pstate->total_tuples = 0; + for (i = 0; i < hashtable->nbatch; ++i) { + ParallelHashJoinBatchAccessor* batch = &hashtable->batches[i]; + + batch->shared->size += batch->size; + batch->shared->estimated_size += batch->estimated_size; + batch->shared->ntuples += batch->ntuples; + batch->shared->old_ntuples += batch->old_ntuples; + batch->size = 0; + batch->estimated_size = 0; + batch->ntuples = 0; + batch->old_ntuples = 0; + pstate->total_tuples += batch->shared->ntuples; + } + LWLockRelease(&pstate->lock); +} + +static void ExecParallelHashIncreaseNumBuckets(HashJoinTable hashtable) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + int i; + HashMemoryChunk chunk; + HashMemoryChunk chunk_s; + + Assert(BarrierPhase(&pstate->build_barrier) == PHJ_BUILD_HASHING_INNER); + + /* + * It's unlikely, but we need to be prepared for new participants to show + * up while we're in the middle of this operation so we need to switch on + * barrier phase here. + */ + switch (PHJ_GROW_BUCKETS_PHASE(BarrierPhase(&pstate->grow_buckets_barrier))) { + case PHJ_GROW_BUCKETS_ELECTING: + /* Elect one participant to prepare to increase nbuckets. */ + if (BarrierArriveAndWait(&pstate->grow_buckets_barrier, WAIT_EVENT_HASH_GROW_BUCKETS_ELECTING)) { + size_t size; + dsa_pointer_atomic* buckets = NULL; + + /* Double the size of the bucket array. */ + pstate->nbuckets *= 2; + size = pstate->nbuckets * sizeof(dsa_pointer_atomic); + hashtable->batches[0].shared->size += size / 2; + dsa_free(hashtable->area, hashtable->batches[0].shared->buckets); + hashtable->batches[0].shared->buckets = dsa_allocate(hashtable->area, size); + buckets = (dsa_pointer_atomic*)dsa_get_address(hashtable->area, hashtable->batches[0].shared->buckets); + for (i = 0; i < pstate->nbuckets; ++i) { + dsa_pointer_atomic_init(&buckets[i], 0); + } + + /* Put the chunk list onto the work queue. */ + pstate->chunk_work_queue = hashtable->batches[0].shared->chunks; + + /* Clear the flag. */ + pstate->growth = PHJ_GROWTH_OK; + } + /* Fall through. */ + + case PHJ_GROW_BUCKETS_ALLOCATING: + /* Wait for the above to complete. */ + (void)BarrierArriveAndWait(&pstate->grow_buckets_barrier, WAIT_EVENT_HASH_GROW_BUCKETS_ALLOCATING); + /* Fall through. */ + + case PHJ_GROW_BUCKETS_REINSERTING: + /* Reinsert all tuples into the hash table. */ + ExecParallelHashEnsureBatchAccessors(hashtable); + ExecParallelHashTableSetCurrentBatch(hashtable, 0); + while ((chunk = ExecParallelHashPopChunkQueue(hashtable, &chunk_s))) { + size_t idx = 0; + while (idx < chunk->used) { + HashJoinTuple hashTuple = (HashJoinTuple)(chunk->data + idx); + int bucketno; + int batchno; + + ExecHashGetBucketAndBatch(hashtable, hashTuple->hashvalue, &bucketno, &batchno); + Assert(batchno == 0); + + /* add the tuple to the proper bucket */ + ExecParallelHashPushTuple(&hashtable->buckets.shared[bucketno], hashTuple, hashTuple->next.shared); + + /* advance index past the tuple */ + idx += MAXALIGN(HJTUPLE_OVERHEAD + HJTUPLE_MINTUPLE(hashTuple)->t_len); + } + + /* allow this loop to be cancellable */ + CHECK_FOR_INTERRUPTS(); + } + (void)BarrierArriveAndWait(&pstate->grow_buckets_barrier, WAIT_EVENT_HASH_GROW_BUCKETS_REINSERTING); + break; + default: + break; + } +} + /* * ExecHashTableInsert * insert a tuple into the hash table depending on the hash value @@ -939,7 +1623,7 @@ void ExecHashTableInsert( /* * put the tuple in hash table */ - HashJoinTuple hashTuple; + HashJoinTuple hashTuple = NULL; int hashTupleSize; /* Create the HashJoinTuple */ @@ -958,8 +1642,8 @@ void ExecHashTableInsert( HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple)); /* Push it onto the front of the bucket's list */ - hashTuple->next = hashtable->buckets[bucketno]; - hashtable->buckets[bucketno] = hashTuple; + hashTuple->next.unshared = hashtable->buckets.unshared[bucketno]; + hashtable->buckets.unshared[bucketno] = hashTuple; /* Record the total width and total tuples for first batch until spill */ if (hashtable->width[0] >= 0) { @@ -1023,8 +1707,9 @@ void ExecHashTableInsert( instrument->memoryinfo.peakOpMemory = hashtable->spaceUsed; } - if (hashtable->width[0] > 0) + if (hashtable->width[0] > 0) { hashtable->width[1] = hashtable->width[1] / hashtable->width[0]; + } hashtable->width[0] = -1; ExecHashIncreaseNumBatches(hashtable); } @@ -1040,6 +1725,81 @@ void ExecHashTableInsert( } } +/* + * ExecHashTableParallelInsert + * insert a tuple into a shared hash table or shared batch tuplestore + */ +void ExecParallelHashTableInsert(HashJoinTable hashtable, TupleTableSlot* slot, uint32 hashvalue) +{ + MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot); + HashJoinTuple shared = NULL; + int bucketno; + int batchno; + +retry: + ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno); + + if (batchno == 0) { + HashJoinTuple hashTuple; + + /* Try to load it into memory. */ + Assert(BarrierPhase(&hashtable->parallel_state->build_barrier) == PHJ_BUILD_HASHING_INNER); + hashTuple = ExecParallelHashTupleAlloc(hashtable, HJTUPLE_OVERHEAD + tuple->t_len, &shared); + if (hashTuple == NULL) { + goto retry; + } + + /* Store the hash value in the HashJoinTuple header. */ + hashTuple->hashvalue = hashvalue; + error_t rc = memcpy_s(HJTUPLE_MINTUPLE(hashTuple), tuple->t_len, tuple, tuple->t_len); + securec_check(rc, "\0", "\0"); + + /* Push it onto the front of the bucket's list */ + + ExecParallelHashPushTuple(&hashtable->buckets.shared[bucketno], hashTuple, shared); + } else { + size_t tuple_size = MAXALIGN(HJTUPLE_OVERHEAD + tuple->t_len); + + Assert(batchno > 0); + + /* Try to preallocate space in the batch if necessary. */ + if (hashtable->batches[batchno].preallocated < tuple_size) { + if (!ExecParallelHashTuplePrealloc(hashtable, batchno, tuple_size)) { + goto retry; + } + } + + Assert(hashtable->batches[batchno].preallocated >= tuple_size); + hashtable->batches[batchno].preallocated -= tuple_size; + sts_puttuple(hashtable->batches[batchno].inner_tuples, &hashvalue, tuple); + } + ++hashtable->batches[batchno].ntuples; +} + +/* + * Insert a tuple into the current hash table. Unlike + * ExecParallelHashTableInsert, this version is not prepared to send the tuple + * to other batches or to run out of memory, and should only be called with + * tuples that belong in the current batch once growth has been disabled. + */ +void ExecParallelHashTableInsertCurrentBatch(HashJoinTable hashtable, TupleTableSlot* slot, uint32 hashvalue) +{ + MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot); + HashJoinTuple hashTuple; + HashJoinTuple shared = NULL; + int batchno; + int bucketno; + + ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno); + Assert(batchno == hashtable->curbatch); + hashTuple = ExecParallelHashTupleAlloc(hashtable, HJTUPLE_OVERHEAD + tuple->t_len, &shared); + hashTuple->hashvalue = hashvalue; + error_t rc = memcpy_s(HJTUPLE_MINTUPLE(hashTuple), tuple->t_len, tuple, tuple->t_len); + securec_check(rc, "\0", "\0"); + HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple)); + ExecParallelHashPushTuple(&hashtable->buckets.shared[bucketno], hashTuple, shared); +} + /* * ExecHashGetHashValue * Compute the hash value for a tuple @@ -1070,10 +1830,11 @@ bool ExecHashGetHashValue(HashJoinTable hashtable, ExprContext* econtext, List* oldContext = MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory); - if (outer_tuple) + if (outer_tuple) { hashfunctions = hashtable->outer_hashfunctions; - else + } else { hashfunctions = hashtable->inner_hashfunctions; + } foreach (hk, hashkeys) { ExprState* keyexpr = (ExprState*)lfirst(hk); @@ -1103,7 +1864,7 @@ bool ExecHashGetHashValue(HashJoinTable hashtable, ExprContext* econtext, List* */ if (isNull) { if (hashtable->hashStrict[i] && !keep_nulls) { - MemoryContextSwitchTo(oldContext); + (void)MemoryContextSwitchTo(oldContext); return false; /* cannot match */ } /* else, leave hashkey unmodified, equivalent to hashcode 0 */ @@ -1118,7 +1879,7 @@ bool ExecHashGetHashValue(HashJoinTable hashtable, ExprContext* econtext, List* i++; } - MemoryContextSwitchTo(oldContext); + (void)MemoryContextSwitchTo(oldContext); hashkey = DatumGetUInt32(hash_uint32(hashkey)); *hashvalue = hashkey; return true; @@ -1139,7 +1900,10 @@ bool ExecHashGetHashValue(HashJoinTable hashtable, ExprContext* econtext, List* * functions are good about randomizing all their output bits, else we are * likely to have very skewed bucket or batch occupancy.) * - * nbuckets doesn't change over the course of the join. + * nbuckets and log2_nbuckets may change while nbatch == 1 because of dynamic + * bucket count growth. Once we start batching, the value is fixed and does + * not change over the course of the join (making it possible to compute batch + * number the way we do here). * * nbatch is always a power of 2; we increase it only by doubling it. This * effectively adds one more bit to the top of the batchno. @@ -1183,12 +1947,13 @@ bool ExecScanHashBucket(HashJoinState* hjstate, ExprContext* econtext) * If the tuple hashed to a skew bucket then scan the skew bucket * otherwise scan the standard hashtable bucket. */ - if (hashTuple != NULL) - hashTuple = hashTuple->next; - else if (hjstate->hj_CurSkewBucketNo != INVALID_SKEW_BUCKET_NO) + if (hashTuple != NULL) { + hashTuple = hashTuple->next.unshared; + } else if (hjstate->hj_CurSkewBucketNo != INVALID_SKEW_BUCKET_NO) { hashTuple = hashtable->skewBucket[hjstate->hj_CurSkewBucketNo]->tuples; - else - hashTuple = hashtable->buckets[hjstate->hj_CurBucketNo]; + } else { + hashTuple = hashtable->buckets.unshared[hjstate->hj_CurBucketNo]; + } while (hashTuple != NULL) { if (hashTuple->hashvalue == hashvalue) { @@ -1214,10 +1979,65 @@ bool ExecScanHashBucket(HashJoinState* hjstate, ExprContext* econtext) * For right Semi/Anti join, we delete mathced tuples in HashTable to make next matching faster, * so pointer hj_PreTuple is designed to follow the hj_CurTuple and to help us to clear the HashTable. */ - if (hjstate->js.jointype == JOIN_RIGHT_SEMI || hjstate->js.jointype == JOIN_RIGHT_ANTI) + if (hjstate->js.jointype == JOIN_RIGHT_SEMI || hjstate->js.jointype == JOIN_RIGHT_ANTI) { hjstate->hj_PreTuple = hashTuple; + } - hashTuple = hashTuple->next; + hashTuple = hashTuple->next.unshared; + } + + /* + * no match + */ + return false; +} + +/* + * ExecParallelScanHashBucket + * scan a hash bucket for matches to the current outer tuple + * + * The current outer tuple must be stored in econtext->ecxt_outertuple. + * + * On success, the inner tuple is stored into hjstate->hj_CurTuple and + * econtext->ecxt_innertuple, using hjstate->hj_HashTupleSlot as the slot + * for the latter. + */ +bool ExecParallelScanHashBucket(HashJoinState* hjstate, ExprContext* econtext) +{ + List* hjclauses = hjstate->hashclauses; + HashJoinTable hashtable = hjstate->hj_HashTable; + HashJoinTuple hashTuple = hjstate->hj_CurTuple; + uint32 hashvalue = hjstate->hj_CurHashValue; + + /* + * hj_CurTuple is the address of the tuple last returned from the current + * bucket, or NULL if it's time to start scanning a new bucket. + */ + if (hashTuple != NULL) { + hashTuple = ExecParallelHashNextTuple(hashtable, hashTuple); + } else { + hashTuple = ExecParallelHashFirstTuple(hashtable, hjstate->hj_CurBucketNo); + } + + while (hashTuple != NULL) { + if (hashTuple->hashvalue == hashvalue) { + TupleTableSlot* inntuple = NULL; + + /* insert hashtable's tuple into exec slot so ExecQual sees it */ + inntuple = + ExecStoreMinimalTuple(HJTUPLE_MINTUPLE(hashTuple), hjstate->hj_HashTupleSlot, false); /* do not pfree */ + econtext->ecxt_innertuple = inntuple; + + /* reset temp memory each time to avoid leaks from qual expr */ + ResetExprContext(econtext); + + if (ExecQual(hjclauses, econtext, false)) { + hjstate->hj_CurTuple = hashTuple; + return true; + } + } + + hashTuple = ExecParallelHashNextTuple(hashtable, hashTuple); } /* @@ -1263,10 +2083,10 @@ bool ExecScanHashTableForUnmatched(HashJoinState* hjstate, ExprContext* econtext * current bucket, or NULL if it's time to start scanning a new * bucket. */ - if (hashTuple != NULL) - hashTuple = hashTuple->next; - else if (hjstate->hj_CurBucketNo < hashtable->nbuckets) { - hashTuple = hashtable->buckets[hjstate->hj_CurBucketNo]; + if (hashTuple != NULL) { + hashTuple = hashTuple->next.unshared; + } else if (hjstate->hj_CurBucketNo < hashtable->nbuckets) { + hashTuple = hashtable->buckets.unshared[hjstate->hj_CurBucketNo]; hjstate->hj_CurBucketNo++; } else if (hjstate->hj_CurSkewBucketNo < hashtable->nSkewBuckets) { int j = hashtable->skewBucketNums[hjstate->hj_CurSkewBucketNo]; @@ -1296,7 +2116,7 @@ bool ExecScanHashTableForUnmatched(HashJoinState* hjstate, ExprContext* econtext return true; } - hashTuple = hashTuple->next; + hashTuple = hashTuple->next.unshared; } } @@ -1324,11 +2144,11 @@ void ExecHashTableReset(HashJoinTable hashtable) oldcxt = MemoryContextSwitchTo(hashtable->batchCxt); /* Reallocate and reinitialize the hash bucket headers. */ - hashtable->buckets = (HashJoinTuple*)palloc0(nbuckets * sizeof(HashJoinTuple)); + hashtable->buckets.unshared = (HashJoinTuple*)palloc0(nbuckets * sizeof(HashJoinTuple)); hashtable->spaceUsed = 0; - MemoryContextSwitchTo(oldcxt); + (void)MemoryContextSwitchTo(oldcxt); /* Forget the chunks (the memory was freed by the context reset above). */ hashtable->chunks = NULL; @@ -1345,8 +2165,9 @@ void ExecHashTableResetMatchFlags(HashJoinTable hashtable) /* Reset all flags in the main table ... */ for (i = 0; i < hashtable->nbuckets; i++) { - for (tuple = hashtable->buckets[i]; tuple != NULL; tuple = tuple->next) + for (tuple = hashtable->buckets.unshared[i]; tuple != NULL; tuple = tuple->next.unshared) { HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(tuple)); + } } /* ... and the same for the skew buckets, if any */ @@ -1354,8 +2175,9 @@ void ExecHashTableResetMatchFlags(HashJoinTable hashtable) int j = hashtable->skewBucketNums[i]; HashSkewBucket* skewBucket = hashtable->skewBucket[j]; - for (tuple = skewBucket->tuples; tuple != NULL; tuple = tuple->next) + for (tuple = skewBucket->tuples; tuple != NULL; tuple = tuple->next.unshared) { HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(tuple)); + } } } @@ -1604,8 +2426,9 @@ static void ExecHashSkewTableInsert(HashJoinTable hashtable, TupleTableSlot* slo HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple)); /* Push it onto the front of the skew bucket's list */ - hashTuple->next = hashtable->skewBucket[bucketNumber]->tuples; + hashTuple->next.unshared = hashtable->skewBucket[bucketNumber]->tuples; hashtable->skewBucket[bucketNumber]->tuples = hashTuple; + Assert(hashTuple != hashTuple->next.unshared); /* Account for space used, and back off if we've used too much */ hashtable->spaceUsed += hashTupleSize; @@ -1651,8 +2474,8 @@ static void ExecHashRemoveNextSkewBucket(HashJoinTable hashtable) /* Process all tuples in the bucket */ hashTuple = bucket->tuples; while (hashTuple != NULL) { - HashJoinTuple nextHashTuple = hashTuple->next; - MinimalTuple tuple; + HashJoinTuple nextHashTuple = hashTuple->next.unshared; + MinimalTuple tuple = NULL; Size tupleSize; /* @@ -1666,8 +2489,8 @@ static void ExecHashRemoveNextSkewBucket(HashJoinTable hashtable) /* Decide whether to put the tuple in the hash table or a temp file */ if (batchno == hashtable->curbatch) { /* Move the tuple to the main hash table */ - hashTuple->next = hashtable->buckets[bucketno]; - hashtable->buckets[bucketno] = hashTuple; + hashTuple->next.unshared = hashtable->buckets.unshared[bucketno]; + hashtable->buckets.unshared[bucketno] = hashTuple; /* We have reduced skew space, but overall space doesn't change */ hashtable->spaceUsedSkew -= tupleSize; } else { @@ -1731,23 +2554,28 @@ static void ExecHashIncreaseBuckets(HashJoinTable hashtable) errno_t rc; /* do nothing if we've decided to shut off growth */ - if (!hashtable->growEnabled) + if (!hashtable->growEnabled) { return; + } /* do nothing if disk spill is already happened */ - if (hashtable->nbatch > 1) + if (hashtable->nbatch > 1) { return; + } /* do nothing if there's still enough space */ - if (hashtable->totalTuples * 2 <= hashtable->nbuckets) + if (hashtable->totalTuples * 2 <= hashtable->nbuckets) { return; + } /* safety check to avoid overflow */ - if ((uint32)hashtable->nbuckets > Min(INT_MAX, MaxAllocSize / (sizeof(HashJoinTuple))) / 2) + if ((uint32)hashtable->nbuckets > Min(INT_MAX, MaxAllocSize / (sizeof(HashJoinTuple))) / 2) { return; + } - hashtable->buckets = (HashJoinTuple*)repalloc(hashtable->buckets, hashtable->nbuckets * 2 * sizeof(HashJoinTuple)); - rc = memset_s(hashtable->buckets + hashtable->nbuckets, + hashtable->buckets.unshared = + (HashJoinTuple*)repalloc(hashtable->buckets.unshared, hashtable->nbuckets * 2 * sizeof(HashJoinTuple)); + rc = memset_s(hashtable->buckets.unshared + hashtable->nbuckets, hashtable->nbuckets * sizeof(HashJoinTuple), 0, hashtable->nbuckets * sizeof(HashJoinTuple)); @@ -1758,21 +2586,22 @@ static void ExecHashIncreaseBuckets(HashJoinTable hashtable) * no longer of the current batch. */ for (int i = 0; i < hashtable->nbuckets; i++) { - HashJoinTuple htuple = hashtable->buckets[i]; + HashJoinTuple htuple = hashtable->buckets.unshared[i]; HashJoinTuple prev = NULL; HashJoinTuple next = NULL; while (htuple != NULL) { int offset = (htuple->hashvalue >> hashtable->log2_nbuckets) & 1; ntotal++; - next = htuple->next; + next = htuple->next.unshared; if (offset == 1) { - if (prev == NULL) - hashtable->buckets[i] = htuple->next; - else + if (prev == NULL) { + hashtable->buckets.unshared[i] = htuple->next.unshared; + } else { prev->next = htuple->next; - htuple->next = hashtable->buckets[i + hashtable->nbuckets]; - hashtable->buckets[i + hashtable->nbuckets] = htuple; + } + htuple->next.unshared = hashtable->buckets.unshared[i + hashtable->nbuckets]; + hashtable->buckets.unshared[i + hashtable->nbuckets] = htuple; Assert((int32)(htuple->hashvalue % (hashtable->nbuckets * 2)) == i + hashtable->nbuckets); nmove++; } else { @@ -1815,7 +2644,7 @@ void ExecHashTableStats(HashJoinTable hashtable, int planid) int maxChainLen = 0; for (int i = 0; i < hashtable->nbuckets; i++) { - HashJoinTuple htuple = hashtable->buckets[i]; + HashJoinTuple htuple = hashtable->buckets.unshared[i]; /* record each hash chain's length and accumulate hash element */ chainLen = 0; @@ -1823,7 +2652,7 @@ void ExecHashTableStats(HashJoinTable hashtable, int planid) while (htuple != NULL) { fillRows++; chainLen++; - htuple = htuple->next; + htuple = htuple->next.unshared; } /* record the number of hash chains with length equal to 1 */ @@ -1888,9 +2717,9 @@ static void* dense_alloc(HashJoinTable hashtable, Size size) */ if (hashtable->chunks != NULL) { newChunk->next = hashtable->chunks->next; - hashtable->chunks->next = newChunk; + hashtable->chunks->next.unshared = newChunk; } else { - newChunk->next = hashtable->chunks; + newChunk->next.unshared = hashtable->chunks; hashtable->chunks = newChunk; } @@ -1913,7 +2742,7 @@ static void* dense_alloc(HashJoinTable hashtable, Size size) newChunk->used = size; newChunk->ntuples = 1; - newChunk->next = hashtable->chunks; + newChunk->next.unshared = hashtable->chunks; hashtable->chunks = newChunk; return newChunk->data; @@ -1927,3 +2756,613 @@ static void* dense_alloc(HashJoinTable hashtable, Size size) /* return pointer to the start of the tuple memory */ return ptr; } + +/* + * Allocate space for a tuple in shared dense storage. This is equivalent to + * dense_alloc but for Parallel Hash using shared memory. + * + * While loading a tuple into shared memory, we might run out of memory and + * decide to repartition, or determine that the load factor is too high and + * decide to expand the bucket array, or discover that another participant has + * commanded us to help do that. Return NULL if number of buckets or batches + * has changed, indicating that the caller must retry (considering the + * possibility that the tuple no longer belongs in the same batch). + */ +static HashJoinTuple ExecParallelHashTupleAlloc(HashJoinTable hashtable, size_t size, HashJoinTuple* shared) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + HashMemoryChunk chunk_shared = NULL; + HashMemoryChunk chunk = NULL; + Size chunk_size; + int curbatch = hashtable->curbatch; + + size = MAXALIGN(size); + + /* + * Fast path: if there is enough space in this backend's current chunk, + * then we can allocate without any locking. + */ + chunk = hashtable->current_chunk; + if (chunk != NULL && size < HASH_CHUNK_THRESHOLD && chunk->maxlen - chunk->used >= size) { + chunk_shared = (HashMemoryChunk)hashtable->current_chunk_shared; + Assert(chunk == dsa_get_address(hashtable->area, chunk_shared)); + HashJoinTuple result = (HashJoinTuple)(chunk->data + chunk->used); + *shared = result; + chunk->used += size; + Assert(chunk->used <= chunk->maxlen); + return result; + } + + /* Slow path: try to allocate a new chunk. */ + (void)LWLockAcquire(&pstate->lock, LW_EXCLUSIVE); + + /* + * Check if we need to help increase the number of buckets or batches. + */ + if (pstate->growth == PHJ_GROWTH_NEED_MORE_BATCHES || pstate->growth == PHJ_GROWTH_NEED_MORE_BUCKETS) { + ParallelHashGrowth growth = pstate->growth; + + hashtable->current_chunk = NULL; + LWLockRelease(&pstate->lock); + + /* Another participant has commanded us to help grow. */ + if (growth == PHJ_GROWTH_NEED_MORE_BATCHES) { + ExecParallelHashIncreaseNumBatches(hashtable); + } else if (growth == PHJ_GROWTH_NEED_MORE_BUCKETS) { + ExecParallelHashIncreaseNumBuckets(hashtable); + } + + /* The caller must retry. */ + return NULL; + } + + /* Oversized tuples get their own chunk. */ + if (size > HASH_CHUNK_THRESHOLD) { + chunk_size = size + HASH_CHUNK_HEADER_SIZE; + } else { + chunk_size = HASH_CHUNK_SIZE; + } + + /* Check if it's time to grow batches or buckets. */ + if (pstate->growth != PHJ_GROWTH_DISABLED) { + Assert(curbatch == 0); + Assert(BarrierPhase(&pstate->build_barrier) == PHJ_BUILD_HASHING_INNER); + + /* + * Check if our space limit would be exceeded. To avoid choking on + * very large tuples or very low work_mem setting, we'll always allow + * each backend to allocate at least one chunk. + */ + if (hashtable->batches[0].at_least_one_chunk && + hashtable->batches[0].shared->size + chunk_size > pstate->space_allowed) { + pstate->growth = PHJ_GROWTH_NEED_MORE_BATCHES; + hashtable->batches[0].shared->space_exhausted = true; + LWLockRelease(&pstate->lock); + + return NULL; + } + + /* Check if our load factor limit would be exceeded. */ + if (hashtable->nbatch == 1) { + hashtable->batches[0].shared->ntuples += hashtable->batches[0].ntuples; + hashtable->batches[0].ntuples = 0; + if (hashtable->batches[0].shared->ntuples + 1 > (size_t)(hashtable->nbuckets * NTUP_PER_BUCKET) && + hashtable->nbuckets < (INT_MAX / 2)) { + pstate->growth = PHJ_GROWTH_NEED_MORE_BUCKETS; + LWLockRelease(&pstate->lock); + + return NULL; + } + } + } + + /* We are cleared to allocate a new chunk. */ + chunk_shared = (HashMemoryChunk)dsa_allocate0(hashtable->area, chunk_size); + hashtable->batches[curbatch].shared->size += chunk_size; + hashtable->batches[curbatch].at_least_one_chunk = true; + + /* Set up the chunk. */ + chunk_shared = chunk_shared; + chunk_shared->maxlen = chunk_size - HASH_CHUNK_HEADER_SIZE; + chunk_shared->used = size; + *shared = (HashJoinTuple)chunk_shared->data; + + /* + * Push it onto the list of chunks, so that it can be found if we need to + * increase the number of buckets or batches (batch 0 only) and later for + * freeing the memory (all batches). + */ + chunk_shared->next.shared = hashtable->batches[curbatch].shared->chunks; + hashtable->batches[curbatch].shared->chunks = chunk_shared; + + if (size <= HASH_CHUNK_THRESHOLD) { + /* + * Make this the current chunk so that we can use the fast path to + * fill the rest of it up in future calls. + */ + hashtable->current_chunk = chunk_shared; + hashtable->current_chunk_shared = chunk_shared; + } + LWLockRelease(&pstate->lock); + Assert(chunk_shared->data == (void*)dsa_get_address(hashtable->area, *shared)); + return (HashJoinTuple)chunk_shared->data; +} + +/* + * One backend needs to set up the shared batch state including tuplestores. + * Other backends will ensure they have correctly configured accessors by + * called ExecParallelHashEnsureBatchAccessors(). + */ +static void ExecParallelHashJoinSetUpBatches(HashJoinTable hashtable, int nbatch) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + ParallelHashJoinBatch* batches = NULL; + MemoryContext oldcxt; + int i; + + Assert(hashtable->batches == NULL); + + /* Allocate space. */ + pstate->batches = (ParallelHashJoinBatch*)MemoryContextAllocZero( + hashtable->area, EstimateParallelHashJoinBatch(hashtable) * nbatch); + pstate->nbatch = nbatch; + batches = (ParallelHashJoinBatch*)dsa_get_address(hashtable->area, pstate->batches); + + /* Use hash join memory context. */ + oldcxt = MemoryContextSwitchTo(hashtable->hashCxt); + + /* Allocate this backend's accessor array. */ + hashtable->nbatch = nbatch; + hashtable->batches = + (ParallelHashJoinBatchAccessor*)palloc0(sizeof(ParallelHashJoinBatchAccessor) * hashtable->nbatch); + + /* Set up the shared state, tuplestores and backend-local accessors. */ + for (i = 0; i < hashtable->nbatch; ++i) { + ParallelHashJoinBatchAccessor* accessor = &hashtable->batches[i]; + ParallelHashJoinBatch* shared = NthParallelHashJoinBatch(batches, i); + char name[MAXPGPATH]; + + /* + * All members of shared were zero-initialized. We just need to set + * up the Barrier. + */ + BarrierInit(&shared->batch_barrier, 0); + if (i == 0) { + /* Batch 0 doesn't need to be loaded. */ + (void)BarrierAttach(&shared->batch_barrier); + while (BarrierPhase(&shared->batch_barrier) < PHJ_BATCH_PROBING) { + (void)BarrierArriveAndWait(&shared->batch_barrier, 0); + } + (void)BarrierDetach(&shared->batch_barrier); + } + + /* Initialize accessor state. All members were zero-initialized. */ + accessor->shared = shared; + + /* Initialize the shared tuplestores. */ + error_t errorno = snprintf_s(name, sizeof(name), sizeof(name) - 1, "i%dof%d", i, hashtable->nbatch); + securec_check_ss_c(errorno, "", ""); + accessor->inner_tuples = sts_initialize(ParallelHashJoinBatchInner(shared), + pstate->nparticipants, + t_thrd.bgworker_cxt.ParallelWorkerNumber + 1, + sizeof(uint32), + SHARED_TUPLESTORE_SINGLE_PASS, + &pstate->fileset, + name); + errorno = snprintf_s(name, sizeof(name), sizeof(name) - 1, "o%dof%d", i, hashtable->nbatch); + securec_check_ss_c(errorno, "", ""); + accessor->outer_tuples = sts_initialize(ParallelHashJoinBatchOuter(shared, pstate->nparticipants), + pstate->nparticipants, + t_thrd.bgworker_cxt.ParallelWorkerNumber + 1, + sizeof(uint32), + SHARED_TUPLESTORE_SINGLE_PASS, + &pstate->fileset, + name); + } + + (void)MemoryContextSwitchTo(oldcxt); +} + +/* + * Free the current set of ParallelHashJoinBatchAccessor objects. + */ +static void ExecParallelHashCloseBatchAccessors(HashJoinTable hashtable) +{ + int i; + + for (i = 0; i < hashtable->nbatch; ++i) { + /* Make sure no files are left open. */ + sts_end_write(hashtable->batches[i].inner_tuples); + sts_end_write(hashtable->batches[i].outer_tuples); + sts_end_parallel_scan(hashtable->batches[i].inner_tuples); + sts_end_parallel_scan(hashtable->batches[i].outer_tuples); + } + pfree(hashtable->batches); + hashtable->batches = NULL; +} + +/* + * Make sure this backend has up-to-date accessors for the current set of + * batches. + */ +static void ExecParallelHashEnsureBatchAccessors(HashJoinTable hashtable) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + ParallelHashJoinBatch* batches = NULL; + MemoryContext oldcxt; + int i; + + if (hashtable->batches != NULL) { + if (hashtable->nbatch == pstate->nbatch) { + return; + } + ExecParallelHashCloseBatchAccessors(hashtable); + } + + /* + * It's possible for a backend to start up very late so that the whole + * join is finished and the shm state for tracking batches has already + * been freed by ExecHashTableDetach(). In that case we'll just leave + * hashtable->batches as NULL so that ExecParallelHashJoinNewBatch() gives + * up early. + */ + if (!DsaPointerIsValid(pstate->batches)) { + return; + } + + /* Use hash join memory context. */ + oldcxt = MemoryContextSwitchTo(hashtable->hashCxt); + + /* Allocate this backend's accessor array. */ + hashtable->nbatch = pstate->nbatch; + hashtable->batches = + (ParallelHashJoinBatchAccessor*)palloc0(sizeof(ParallelHashJoinBatchAccessor) * hashtable->nbatch); + + /* Find the base of the pseudo-array of ParallelHashJoinBatch objects. */ + batches = (ParallelHashJoinBatch*)dsa_get_address(hashtable->area, pstate->batches); + + /* Set up the accessor array and attach to the tuplestores. */ + for (i = 0; i < hashtable->nbatch; ++i) { + ParallelHashJoinBatchAccessor* accessor = &hashtable->batches[i]; + ParallelHashJoinBatch* shared = NthParallelHashJoinBatch(batches, i); + + accessor->shared = shared; + accessor->preallocated = 0; + accessor->done = false; + accessor->inner_tuples = sts_attach( + ParallelHashJoinBatchInner(shared), t_thrd.bgworker_cxt.ParallelWorkerNumber + 1, &pstate->fileset); + accessor->outer_tuples = sts_attach(ParallelHashJoinBatchOuter(shared, pstate->nparticipants), + t_thrd.bgworker_cxt.ParallelWorkerNumber + 1, + &pstate->fileset); + } + + (void)MemoryContextSwitchTo(oldcxt); +} + +/* + * Allocate an empty shared memory hash table for a given batch. + */ +void ExecParallelHashTableAlloc(HashJoinTable hashtable, int batchno) +{ + ParallelHashJoinBatch* batch = hashtable->batches[batchno].shared; + dsa_pointer_atomic* buckets = NULL; + int nbuckets = hashtable->parallel_state->nbuckets; + int i; + + batch->buckets = dsa_allocate(hashtable->area, sizeof(dsa_pointer_atomic) * nbuckets); + buckets = (dsa_pointer_atomic*)dsa_get_address(hashtable->area, batch->buckets); + for (i = 0; i < nbuckets; ++i) { + buckets[i] = 0; + } +} + +/* + * If we are currently attached to a shared hash join batch, detach. If we + * are last to detach, clean up. + */ +void ExecHashTableDetachBatch(HashJoinTable hashtable) +{ + if (hashtable->parallel_state != NULL && hashtable->curbatch >= 0) { + int curbatch = hashtable->curbatch; + ParallelHashJoinBatch* batch = hashtable->batches[curbatch].shared; + + /* Make sure any temporary files are closed. */ + sts_end_parallel_scan(hashtable->batches[curbatch].inner_tuples); + sts_end_parallel_scan(hashtable->batches[curbatch].outer_tuples); + + /* Detach from the batch we were last working on. */ + if (BarrierArriveAndDetach(&batch->batch_barrier)) { + /* + * Technically we shouldn't access the barrier because we're no + * longer attached, but since there is no way it's moving after + * this point it seems safe to make the following assertion. + */ + Assert(BarrierPhase(&batch->batch_barrier) == PHJ_BATCH_DONE); + + /* Free shared chunks and buckets. */ + while (DsaPointerIsValid(batch->chunks)) { + HashMemoryChunk next = batch->chunks->next.shared; + dsa_free(hashtable->area, batch->chunks); + batch->chunks = next; + } + if (DsaPointerIsValid(batch->buckets)) { + dsa_free(hashtable->area, batch->buckets); + batch->buckets = InvalidDsaPointer; + } + } + ExecParallelHashUpdateSpacePeak(hashtable, curbatch); + /* Remember that we are not attached to a batch. */ + hashtable->curbatch = -1; + } +} + +/* + * Detach from all shared resources. If we are last to detach, clean up. + */ +void ExecHashTableDetach(HashJoinTable hashtable) +{ + if (hashtable->parallel_state) { + ParallelHashJoinState* pstate = hashtable->parallel_state; + int i; + + /* Make sure any temporary files are closed. */ + if (hashtable->batches) { + for (i = 0; i < hashtable->nbatch; ++i) { + sts_end_write(hashtable->batches[i].inner_tuples); + sts_end_write(hashtable->batches[i].outer_tuples); + sts_end_parallel_scan(hashtable->batches[i].inner_tuples); + sts_end_parallel_scan(hashtable->batches[i].outer_tuples); + } + } + + /* If we're last to detach, clean up shared memory. */ + if (BarrierDetach(&pstate->build_barrier)) { + if (DsaPointerIsValid(pstate->batches)) { + dsa_free(hashtable->area, pstate->batches); + pstate->batches = InvalidDsaPointer; + } + } + + hashtable->parallel_state = NULL; + } +} + +/* + * Get the first tuple in a given bucket identified by number. + */ +static inline HashJoinTuple ExecParallelHashFirstTuple(HashJoinTable hashtable, int bucketno) +{ + HashJoinTuple tuple; + ParallelHashJoinState* parallelState = hashtable->parallel_state; + Assert(parallelState); + tuple = (HashJoinTuple)dsa_get_address(hashtable->area, hashtable->buckets.shared[bucketno]); + return tuple; +} + +/* + * Get the next tuple in the same bucket as 'tuple'. + */ +static inline HashJoinTuple ExecParallelHashNextTuple(HashJoinTable hashtable, HashJoinTuple tuple) +{ + Assert(hashtable->parallel_state); + return tuple->next.shared; +} + +/* + * Insert a tuple at the front of a chain of tuples in DSA memory atomically. + */ +static inline void ExecParallelHashPushTuple(HashJoinTuple* head, HashJoinTuple tuple, HashJoinTuple tuple_shared) +{ + for (;;) { + tuple->next.shared = *head; + if (pg_atomic_compare_exchange_uintptr( + (uintptr_t*)head, (uintptr_t*)&(tuple->next.shared), (uintptr_t)tuple_shared)) { + break; + } + } +} + +/* + * Prepare to work on a given batch. + */ +void ExecParallelHashTableSetCurrentBatch(HashJoinTable hashtable, int batchno) +{ + Assert(hashtable->batches[batchno].shared->buckets != InvalidDsaPointer); + + hashtable->curbatch = batchno; + hashtable->buckets.shared = + (HashJoinTuple*)dsa_get_address(hashtable->area, hashtable->batches[batchno].shared->buckets); + hashtable->nbuckets = hashtable->parallel_state->nbuckets; + hashtable->log2_nbuckets = my_log2(hashtable->nbuckets); + hashtable->current_chunk = NULL; + hashtable->current_chunk_shared = InvalidDsaPointer; + hashtable->batches[batchno].at_least_one_chunk = false; +} + +/* + * Take the next available chunk from the queue of chunks being worked on in + * parallel. Return NULL if there are none left. Otherwise return a pointer + * to the chunk, and set *shared to the DSA pointer to the chunk. + */ +static HashMemoryChunk ExecParallelHashPopChunkQueue(HashJoinTable hashtable, HashMemoryChunk* shared) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + HashMemoryChunk chunk; + + (void)LWLockAcquire(&pstate->lock, LW_EXCLUSIVE); + if (DsaPointerIsValid(pstate->chunk_work_queue)) { + *shared = pstate->chunk_work_queue; + chunk = *shared; + pstate->chunk_work_queue = chunk->next.shared; + } else { + chunk = NULL; + } + LWLockRelease(&pstate->lock); + + return chunk; +} + +/* + * Increase the space preallocated in this backend for a given inner batch by + * at least a given amount. This allows us to track whether a given batch + * would fit in memory when loaded back in. Also increase the number of + * batches or buckets if required. + * + * This maintains a running estimation of how much space will be taken when we + * load the batch back into memory by simulating the way chunks will be handed + * out to workers. It's not perfectly accurate because the tuples will be + * packed into memory chunks differently by ExecParallelHashTupleAlloc(), but + * it should be pretty close. It tends to overestimate by a fraction of a + * chunk per worker since all workers gang up to preallocate during hashing, + * but workers tend to reload batches alone if there are enough to go around, + * leaving fewer partially filled chunks. This effect is bounded by + * nparticipants. + * + * Return false if the number of batches or buckets has changed, and the + * caller should reconsider which batch a given tuple now belongs in and call + * again. + */ +static bool ExecParallelHashTuplePrealloc(HashJoinTable hashtable, int batchno, size_t size) +{ + ParallelHashJoinState* pstate = hashtable->parallel_state; + ParallelHashJoinBatchAccessor* batch = &hashtable->batches[batchno]; + size_t want = Max(size, HASH_CHUNK_SIZE - HASH_CHUNK_HEADER_SIZE); + + Assert(batchno > 0); + Assert(batchno < hashtable->nbatch); + + (void)LWLockAcquire(&pstate->lock, LW_EXCLUSIVE); + + /* Has another participant commanded us to help grow? */ + if (pstate->growth == PHJ_GROWTH_NEED_MORE_BATCHES || pstate->growth == PHJ_GROWTH_NEED_MORE_BUCKETS) { + ParallelHashGrowth growth = pstate->growth; + + LWLockRelease(&pstate->lock); + if (growth == PHJ_GROWTH_NEED_MORE_BATCHES) { + ExecParallelHashIncreaseNumBatches(hashtable); + } else if (growth == PHJ_GROWTH_NEED_MORE_BUCKETS) { + ExecParallelHashIncreaseNumBuckets(hashtable); + } + + return false; + } + + if (pstate->growth != PHJ_GROWTH_DISABLED && batch->at_least_one_chunk && + (batch->shared->estimated_size + size > pstate->space_allowed)) { + /* + * We have determined that this batch would exceed the space budget if + * loaded into memory. Command all participants to help repartition. + */ + batch->shared->space_exhausted = true; + pstate->growth = PHJ_GROWTH_NEED_MORE_BATCHES; + LWLockRelease(&pstate->lock); + + return false; + } + + batch->at_least_one_chunk = true; + batch->shared->estimated_size += want + HASH_CHUNK_HEADER_SIZE; + batch->preallocated = want; + LWLockRelease(&pstate->lock); + + return true; +} + +/* + * Set up a space in the DSM for all workers to record instrumentation data + * about their hash table. + */ +void ExecHashInitializeDSM(HashState* node, ParallelContext* pcxt, int nodeid) +{ + int plan_node_id = node->ps.plan->plan_node_id; + knl_u_parallel_context* cxt = (knl_u_parallel_context*)pcxt->seg; + size_t size = offsetof(SharedHashInfo, instrument) + pcxt->nworkers * sizeof(Instrumentation); + node->shared_info = (SharedHashInfo*)MemoryContextAllocZero(cxt->memCtx, size); + node->shared_info->num_workers = pcxt->nworkers; + node->shared_info->plan_node_id = plan_node_id; + cxt->pwCtx->queryInfo.shared_info[nodeid] = node->shared_info; +} + +/* + * Locate the DSM space for hash table instrumentation data that we'll write + * to at shutdown time. + */ +void ExecHashInitializeWorker(HashState* node, void* pwcxt) +{ + knl_u_parallel_context* cxt = (knl_u_parallel_context*)pwcxt; + int planNodeId = node->ps.plan->plan_node_id; + + SharedHashInfo* shared_info = NULL; + for (int i = 0; i < cxt->pwCtx->queryInfo.hash_num; i++) { + if (planNodeId == ((SharedHashInfo*)(cxt->pwCtx->queryInfo.shared_info[i]))->plan_node_id) { + shared_info = (SharedHashInfo*)cxt->pwCtx->queryInfo.shared_info[i]; + break; + } + } + if (shared_info == NULL) { + ereport(ERROR, (errmsg("could not find plan info, plan node id:%d", planNodeId))); + } + + node->instrument = &shared_info->instrument[t_thrd.bgworker_cxt.ParallelWorkerNumber]; +} + +/* + * Update this backend's copy of hashtable->spacePeak to account for a given + * batch. This is called at the end of hashing for batch 0, and then for each + * batch when it is done or discovered to be already done. The result is used + * for EXPLAIN output. + */ +void ExecParallelHashUpdateSpacePeak(HashJoinTable hashtable, int batchno) +{ + int64 size; + + size = (int64)hashtable->batches[batchno].shared->size; + size += sizeof(dsa_pointer_atomic) * (int64)hashtable->nbuckets; + hashtable->spacePeak = Max(hashtable->spacePeak, size); +} + +/* + * Retrieve instrumentation data from workers before the DSM segment is + * detached, so that EXPLAIN can access it. + */ +void ExecHashRetrieveInstrumentation(HashState* node) +{ + SharedHashInfo* sharedHashInfo = node->shared_info; + /* Replace node->shared_info with a copy in backend-local memory. */ + size_t size = offsetof(SharedHashInfo, instrument) + sharedHashInfo->num_workers * sizeof(Instrumentation); + node->shared_info = (SharedHashInfo*)palloc(size); + error_t rc = memcpy_s(node->shared_info, size, sharedHashInfo, size); + securec_check(rc, "", ""); +} + +/* + * Copy the instrumentation data from 'hashtable' into a HashInstrumentation + * struct. + */ +void ExecHashGetInstrumentation(Instrumentation* instrument, HashJoinTable hashtable) +{ + instrument->sorthashinfo.nbatch = hashtable->nbatch; + instrument->sorthashinfo.nbuckets = hashtable->nbuckets; + instrument->sorthashinfo.nbatch_original = hashtable->nbatch_original; + instrument->sorthashinfo.spacePeak = hashtable->spacePeak; + if (hashtable->width[0] > 0) { + hashtable->width[1] = hashtable->width[1] / hashtable->width[0]; + hashtable->width[0] = -1; + } + instrument->width = (int)hashtable->width[1]; + instrument->sysBusy = hashtable->causedBySysRes; + instrument->spreadNum = hashtable->spreadNum; +} + +/* + * Copy instrumentation data from this worker's hash table (if it built one) + * to DSM memory so the leader can retrieve it. This must be done in an + * ExecShutdownHash() rather than ExecEndHash() because the latter runs after + * we've detached from the DSM segment. + */ +void ExecShutdownHash(HashState* node) +{ + if (node->instrument && node->hashtable) { + ExecHashGetInstrumentation(node->instrument, node->hashtable); + } +} diff --git a/src/gausskernel/runtime/executor/nodeHashjoin.cpp b/src/gausskernel/runtime/executor/nodeHashjoin.cpp index 83ad7b429..572c0ee67 100755 --- a/src/gausskernel/runtime/executor/nodeHashjoin.cpp +++ b/src/gausskernel/runtime/executor/nodeHashjoin.cpp @@ -11,11 +11,104 @@ * IDENTIFICATION * src/gausskernel/runtime/executor/nodeHashjoin.cpp * - * ------------------------------------------------------------------------- + * PARALLELISM + * + * Hash joins can participate in parallel query execution in several ways. A + * parallel-oblivious hash join is one where the node is unaware that it is + * part of a parallel plan. In this case, a copy of the inner plan is used to + * build a copy of the hash table in every backend, and the outer plan could + * either be built from a partial or complete path, so that the results of the + * hash join are correspondingly either partial or complete. A parallel-aware + * hash join is one that behaves differently, coordinating work between + * backends, and appears as Parallel Hash Join in EXPLAIN output. A Parallel + * Hash Join always appears with a Parallel Hash node. + * + * Parallel-aware hash joins use the same per-backend state machine to track + * progress through the hash join algorithm as parallel-oblivious hash joins. + * In a parallel-aware hash join, there is also a shared state machine that + * co-operating backends use to synchronize their local state machines and + * program counters. The shared state machine is managed with a Barrier IPC + * primitive. When all attached participants arrive at a barrier, the phase + * advances and all waiting participants are released. + * + * When a participant begins working on a parallel hash join, it must first + * figure out how much progress has already been made, because participants + * don't wait for each other to begin. For this reason there are switch + * statements at key points in the code where we have to synchronize our local + * state machine with the phase, and then jump to the correct part of the + * algorithm so that we can get started. + * + * One barrier called build_barrier is used to coordinate the hashing phases. + * The phase is represented by an integer which begins at zero and increments + * one by one, but in the code it is referred to by symbolic names as follows: + * + * PHJ_BUILD_ELECTING -- initial state + * PHJ_BUILD_ALLOCATING -- one sets up the batches and table 0 + * PHJ_BUILD_HASHING_INNER -- all hash the inner rel + * PHJ_BUILD_HASHING_OUTER -- (multi-batch only) all hash the outer + * PHJ_BUILD_DONE -- building done, probing can begin + * + * While in the phase PHJ_BUILD_HASHING_INNER a separate pair of barriers may + * be used repeatedly as required to coordinate expansions in the number of + * batches or buckets. Their phases are as follows: + * + * PHJ_GROW_BATCHES_ELECTING -- initial state + * PHJ_GROW_BATCHES_ALLOCATING -- one allocates new batches + * PHJ_GROW_BATCHES_REPARTITIONING -- all repartition + * PHJ_GROW_BATCHES_FINISHING -- one cleans up, detects skew + * + * PHJ_GROW_BUCKETS_ELECTING -- initial state + * PHJ_GROW_BUCKETS_ALLOCATING -- one allocates new buckets + * PHJ_GROW_BUCKETS_REINSERTING -- all insert tuples + * + * If the planner got the number of batches and buckets right, those won't be + * necessary, but on the other hand we might finish up needing to expand the + * buckets or batches multiple times while hashing the inner relation to stay + * within our memory budget and load factor target. For that reason it's a + * separate pair of barriers using circular phases. + * + * The PHJ_BUILD_HASHING_OUTER phase is required only for multi-batch joins, + * because we need to divide the outer relation into batches up front in order + * to be able to process batches entirely independently. In contrast, the + * parallel-oblivious algorithm simply throws tuples 'forward' to 'later' + * batches whenever it encounters them while scanning and probing, which it + * can do because it processes batches in serial order. + * + * Once PHJ_BUILD_DONE is reached, backends then split up and process + * different batches, or gang up and work together on probing batches if there + * aren't enough to go around. For each batch there is a separate barrier + * with the following phases: + * + * PHJ_BATCH_ELECTING -- initial state + * PHJ_BATCH_ALLOCATING -- one allocates buckets + * PHJ_BATCH_LOADING -- all load the hash table from disk + * PHJ_BATCH_PROBING -- all probe + * PHJ_BATCH_DONE -- end + * + * Batch 0 is a special case, because it starts out in phase + * PHJ_BATCH_PROBING; populating batch 0's hash table is done during + * PHJ_BUILD_HASHING_INNER so we can skip loading. + * + * Initially we try to plan for a single-batch hash join using the combined + * hash_mem of all participants to create a large shared hash table. If that + * turns out either at planning or execution time to be impossible then we + * fall back to regular hash_mem sized hash tables. + * + * To avoid deadlocks, we never wait for any barrier unless it is known that + * all other backends attached to it are actively executing the node or have + * already arrived. Practically, that means that we never return a tuple + * while attached to a barrier, unless the barrier has reached its final + * state. In the slightly special case of the per-batch barrier, we return + * tuples while in PHJ_BATCH_PROBING phase, but that's OK because we use + * BarrierArriveAndDetach() to advance it to PHJ_BATCH_DONE without waiting. + * + *------------------------------------------------------------------------- */ + #include "postgres.h" #include "knl/knl_variable.h" +#include "access/parallel.h" #include "executor/executor.h" #include "executor/execStream.h" #include "executor/hashjoin.h" @@ -24,6 +117,7 @@ #include "miscadmin.h" #include "utils/anls_opt.h" #include "utils/memutils.h" +#include "utils/sharedtuplestore.h" /* * States of the ExecHashJoin state machine @@ -34,7 +128,6 @@ #define HJ_FILL_OUTER_TUPLE 4 #define HJ_FILL_INNER_TUPLES 5 #define HJ_NEED_NEW_BATCH 6 - /* Returns true if doing null-fill on outer relation */ #define HJ_FILL_OUTER(hjstate) ((hjstate)->hj_NullInnerTupleSlot != NULL) /* Returns true if doing null-fill on inner relation */ @@ -43,19 +136,13 @@ static TupleTableSlot* ExecHashJoinOuterGetTuple(PlanState* outerNode, HashJoinState* hjstate, uint32* hashvalue); static TupleTableSlot* ExecHashJoinGetSavedTuple( HashJoinState* hjstate, BufFile* file, uint32* hashvalue, TupleTableSlot* tupleSlot); +static TupleTableSlot* ExecParallelHashJoinOuterGetTuple( + PlanState* outerNode, HashJoinState* hjstate, uint32* hashvalue); static bool ExecHashJoinNewBatch(HashJoinState* hjstate); +static bool ExecParallelHashJoinNewBatch(HashJoinState* hjstate); +static void ExecParallelHashJoinPartitionOuter(HashJoinState* node); -/* ---------------------------------------------------------------- - * ExecHashJoin - * - * This function implements the Hybrid Hashjoin algorithm. - * - * Note: the relation we build hash table on is the "inner" - * the other one is "outer". - * ---------------------------------------------------------------- - */ -/* return: a tuple or NULL */ -TupleTableSlot* ExecHashJoin(HashJoinState* node) +FORCE_INLINE static TupleTableSlot* ExecHashJoinImpl(HashJoinState* node, bool parallel) { PlanState* outerNode = NULL; HashState* hashNode = NULL; @@ -63,12 +150,13 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) List* otherqual = NIL; ExprContext* econtext = NULL; ExprDoneCond isDone; - HashJoinTable hashtable; + HashJoinTable hashtable = NULL; TupleTableSlot* outerTupleSlot = NULL; uint32 hashvalue; int batchno; - MemoryContext oldcxt; + MemoryContext oldcxt = NULL; JoinType jointype; + ParallelHashJoinState* parallel_state = NULL; /* * get information from HashJoin node @@ -80,6 +168,7 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) hashtable = node->hj_HashTable; econtext = node->js.ps.ps_ExprContext; jointype = node->js.jointype; + parallel_state = hashNode->parallel_state; /* * Check to see if we're still projecting out tuples from a previous join @@ -90,8 +179,9 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) TupleTableSlot* result = NULL; result = ExecProject(node->js.ps.ps_ProjInfo, &isDone); - if (isDone == ExprMultipleResult) + if (isDone == ExprMultipleResult) { return result; + } /* Done with that source tuple... */ node->js.ps.ps_TupFromTlist = false; } @@ -140,6 +230,16 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) if (HJ_FILL_INNER(node)) { /* no chance to not build the hash table */ node->hj_FirstOuterTupleSlot = NULL; + } else if (parallel) { + /* + * The empty-outer optimization is not implemented for + * shared hash tables, because no one participant can + * determine that there are no outer tuples, and it's not + * yet clear that it's worth the synchronization overhead + * of reaching consensus to figure that out. So we have + * to build the hash table. + */ + node->hj_FirstOuterTupleSlot = NULL; } else if ((HJ_FILL_OUTER(node) || (outerNode->plan->startup_cost < hashNode->ps.plan->total_cost && !node->hj_OuterNotEmpty)) && !node->hj_streamBothSides) { @@ -162,23 +262,28 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) (node->js.ps.plan)->plan_node_id, getSessionMemoryUsageMB())); return NULL; - } else + } else { node->hj_OuterNotEmpty = true; - } else + } + } else { node->hj_FirstOuterTupleSlot = NULL; + } /* - * create the hash table, sometimes we should keep nulls + * Create the hash table. If using Parallel Hash, then + * whoever gets here first will create the hash table and any + * later arrivals will merely attach to it. */ oldcxt = MemoryContextSwitchTo(hashNode->ps.nodeContext); - hashtable = ExecHashTableCreate((Hash*)hashNode->ps.plan, - node->hj_HashOperators, - HJ_FILL_INNER(node) || node->js.nulleqqual != NIL); + hashtable = ExecHashTableCreate( + hashNode, node->hj_HashOperators, HJ_FILL_INNER(node) || node->js.nulleqqual != NIL); MemoryContextSwitchTo(oldcxt); node->hj_HashTable = hashtable; /* - * execute the Hash node, to build the hash table + * Execute the Hash node, to build the hash table. If using + * Parallel Hash, then we'll try to help hashing unless we + * arrived too late. */ WaitState oldStatus = pgstat_report_waitstatus(STATE_EXEC_HASHJOIN_BUILD_HASH); hashNode->hashtable = hashtable; @@ -223,8 +328,32 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) * immediately set it again.) */ node->hj_OuterNotEmpty = false; + if (parallel) { + Barrier* build_barrier = NULL; - node->hj_JoinState = HJ_NEED_NEW_OUTER; + build_barrier = ¶llel_state->build_barrier; + Assert(BarrierPhase(build_barrier) == PHJ_BUILD_HASHING_OUTER || + BarrierPhase(build_barrier) == PHJ_BUILD_DONE); + if (BarrierPhase(build_barrier) == PHJ_BUILD_HASHING_OUTER) { + /* + * If multi-batch, we need to hash the outer relation + * up front. + */ + if (hashtable->nbatch > 1) { + ExecParallelHashJoinPartitionOuter(node); + } + (void)BarrierArriveAndWait(build_barrier, WAIT_EVENT_HASH_BUILD_HASHING_OUTER); + } + Assert(BarrierPhase(build_barrier) == PHJ_BUILD_DONE); + + /* Each backend should now select a batch to work on. */ + hashtable->curbatch = -1; + node->hj_JoinState = HJ_NEED_NEW_BATCH; + + continue; + } else { + node->hj_JoinState = HJ_NEED_NEW_OUTER; + } } /* fall through */ case HJ_NEED_NEW_OUTER: @@ -232,15 +361,20 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) /* * We don't have an outer tuple, try to get the next one */ - outerTupleSlot = ExecHashJoinOuterGetTuple(outerNode, node, &hashvalue); + if (parallel) { + outerTupleSlot = ExecParallelHashJoinOuterGetTuple(outerNode, node, &hashvalue); + } else { + outerTupleSlot = ExecHashJoinOuterGetTuple(outerNode, node, &hashvalue); + } if (TupIsNull(outerTupleSlot)) { /* end of batch, or maybe whole join */ if (HJ_FILL_INNER(node)) { /* set up to scan for unmatched inner tuples */ ExecPrepHashTableForUnmatched(node); node->hj_JoinState = HJ_FILL_INNER_TUPLES; - } else + } else { node->hj_JoinState = HJ_NEED_NEW_BATCH; + } continue; } @@ -265,12 +399,12 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) * Need to postpone this outer tuple to a later batch. * Save it in the corresponding outer-batch file. */ + Assert(parallel_state == NULL); Assert(batchno > hashtable->curbatch); MinimalTuple tuple = ExecFetchSlotMinimalTuple(outerTupleSlot); ExecHashJoinSaveTuple(tuple, hashvalue, &hashtable->outerBatchFile[batchno]); *hashtable->spill_size += sizeof(uint32) + tuple->t_len; pgstat_increase_session_spill_size(sizeof(uint32) + tuple->t_len); - /* Loop around, staying in HJ_NEED_NEW_OUTER state */ continue; } @@ -279,8 +413,9 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) node->hj_JoinState = HJ_SCAN_BUCKET; /* Prepare for the clear-process if necessary */ - if (jointype == JOIN_RIGHT_ANTI || jointype == JOIN_RIGHT_SEMI) + if (jointype == JOIN_RIGHT_ANTI || jointype == JOIN_RIGHT_SEMI) { node->hj_PreTuple = NULL; + } /* fall through */ case HJ_SCAN_BUCKET: @@ -295,10 +430,18 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) /* * Scan the selected hash bucket for matches to current outer */ - if (!ExecScanHashBucket(node, econtext)) { - /* out of matches; check for possible outer-join fill */ - node->hj_JoinState = HJ_FILL_OUTER_TUPLE; - continue; + if (parallel) { + if (!ExecParallelScanHashBucket(node, econtext)) { + /* out of matches; check for possible outer-join fill */ + node->hj_JoinState = HJ_FILL_OUTER_TUPLE; + continue; + } + } else { + if (!ExecScanHashBucket(node, econtext)) { + /* out of matches; check for possible outer-join fill */ + node->hj_JoinState = HJ_FILL_OUTER_TUPLE; + continue; + } } /* @@ -323,14 +466,17 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) */ if (jointype == JOIN_RIGHT_ANTI || jointype == JOIN_RIGHT_SEMI || jointype == JOIN_RIGHT_ANTI_FULL) { - if (node->hj_PreTuple) + if (node->hj_PreTuple) { node->hj_PreTuple->next = node->hj_CurTuple->next; - else if (node->hj_CurSkewBucketNo != INVALID_SKEW_BUCKET_NO) - hashtable->skewBucket[node->hj_CurSkewBucketNo]->tuples = node->hj_CurTuple->next; - else - hashtable->buckets[node->hj_CurBucketNo] = node->hj_CurTuple->next; - if (jointype == JOIN_RIGHT_ANTI || jointype == JOIN_RIGHT_ANTI_FULL) + } else if (node->hj_CurSkewBucketNo != INVALID_SKEW_BUCKET_NO) { + hashtable->skewBucket[node->hj_CurSkewBucketNo]->tuples = + (HashJoinTupleData*)(node->hj_CurTuple->next.unshared); + } else { + hashtable->buckets.unshared[node->hj_CurBucketNo] = node->hj_CurTuple->next.unshared; + } + if (jointype == JOIN_RIGHT_ANTI || jointype == JOIN_RIGHT_ANTI_FULL) { continue; + } } else { HeapTupleHeaderSetMatch(HJTUPLE_MINTUPLE(node->hj_CurTuple)); @@ -342,8 +488,9 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) /* Semi join: we'll consider returning the first match, but after * that we're done with this outer tuple */ - if (jointype == JOIN_SEMI) + if (jointype == JOIN_SEMI) { node->hj_JoinState = HJ_NEED_NEW_OUTER; + } } if (otherqual == NIL || ExecQual(otherqual, econtext, false)) { @@ -355,13 +502,15 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) node->js.ps.ps_TupFromTlist = (isDone == ExprMultipleResult); return result; } - } else + } else { InstrCountFiltered2(node, 1); + } } else { InstrCountFiltered1(node, 1); /* For right Semi/Anti join, we set hj_PreTuple following hj_CurTuple */ - if (jointype == JOIN_RIGHT_ANTI || jointype == JOIN_RIGHT_SEMI) + if (jointype == JOIN_RIGHT_ANTI || jointype == JOIN_RIGHT_SEMI) { node->hj_PreTuple = node->hj_CurTuple; + } } break; @@ -390,8 +539,9 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) node->js.ps.ps_TupFromTlist = (isDone == ExprMultipleResult); return result; } - } else + } else { InstrCountFiltered2(node, 1); + } } break; @@ -423,8 +573,9 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) node->js.ps.ps_TupFromTlist = (isDone == ExprMultipleResult); return result; } - } else + } else { InstrCountFiltered2(node, 1); + } break; case HJ_NEED_NEW_BATCH: @@ -432,16 +583,23 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) /* * Try to advance to next batch. Done if there are no more. */ - if (!ExecHashJoinNewBatch(node)) { - ExecEarlyFree(outerPlanState(node)); - EARLY_FREE_LOG(elog(LOG, - "Early Free: HashJoin Probe is done" - " at node %d, memory used %d MB.", - (node->js.ps.plan)->plan_node_id, - getSessionMemoryUsageMB())); + if (parallel) { + if (!ExecParallelHashJoinNewBatch(node)) { + return NULL; /* end of parallel-aware join */ + } + } else { + if (!ExecHashJoinNewBatch(node)) { + ExecEarlyFree(outerPlanState(node)); + EARLY_FREE_LOG(elog(LOG, + "Early Free: HashJoin Probe is done" + " at node %d, memory used %d MB.", + (node->js.ps.plan)->plan_node_id, + getSessionMemoryUsageMB())); - return NULL; /* end of join */ + return NULL; /* end of join */ + } } + node->hj_JoinState = HJ_NEED_NEW_OUTER; break; @@ -454,6 +612,25 @@ TupleTableSlot* ExecHashJoin(HashJoinState* node) } } +/* ---------------------------------------------------------------- + * ExecHashJoin + * + * This function implements the Hybrid Hashjoin algorithm. + * + * Note: the relation we build hash table on is the "inner" + * the other one is "outer". + * ---------------------------------------------------------------- + */ +/* return: a tuple or NULL */ +TupleTableSlot* ExecHashJoin(HashJoinState* node) +{ + /* + * On sufficiently smart compilers this should be inlined with the + * parallel-aware branches removed. + */ + return ExecHashJoinImpl(node, node->isParallel); +} + /* ---------------------------------------------------------------- * ExecInitHashJoin * @@ -650,7 +827,7 @@ void ExecEndHashJoin(HashJoinState* node) /* * ExecHashJoinOuterGetTuple * - * get the next outer tuple for hashjoin: either by + * get the next outer tuple for a parallel oblivious hashjoin: either by * executing the outer plan node in the first pass, or from * the temp files for the hashjoin batches. * @@ -671,10 +848,11 @@ static TupleTableSlot* ExecHashJoinOuterGetTuple(PlanState* outerNode, HashJoinS * ExecHashJoin() and not used yet. */ slot = hjstate->hj_FirstOuterTupleSlot; - if (!TupIsNull(slot)) + if (!TupIsNull(slot)) { hjstate->hj_FirstOuterTupleSlot = NULL; - else + } else { slot = ExecProcNode(outerNode); + } while (!TupIsNull(slot)) { /* @@ -684,11 +862,11 @@ static TupleTableSlot* ExecHashJoinOuterGetTuple(PlanState* outerNode, HashJoinS econtext->ecxt_outertuple = slot; if (ExecHashGetHashValue(hashtable, - econtext, - hjstate->hj_OuterHashKeys, - true, /* outer tuple */ - HJ_FILL_OUTER(hjstate) || hjstate->js.nulleqqual != NIL, /* compute null ? */ - hashvalue)) { + econtext, + hjstate->hj_OuterHashKeys, + true, /* outer tuple */ + HJ_FILL_OUTER(hjstate) || hjstate->js.nulleqqual != NIL, /* compute null ? */ + hashvalue)) { /* remember outer relation is not empty for possible rescan */ hjstate->hj_OuterNotEmpty = true; @@ -708,12 +886,67 @@ static TupleTableSlot* ExecHashJoinOuterGetTuple(PlanState* outerNode, HashJoinS * In outer-join cases, we could get here even though the batch file * is empty. */ - if (file == NULL) + if (file == NULL) { return NULL; + } slot = ExecHashJoinGetSavedTuple(hjstate, file, hashvalue, hjstate->hj_OuterTupleSlot); - if (!TupIsNull(slot)) + if (!TupIsNull(slot)) { return slot; + } + } + + /* End of this batch */ + return NULL; +} + +/* + * ExecHashJoinOuterGetTuple variant for the parallel case. + */ +static TupleTableSlot* ExecParallelHashJoinOuterGetTuple( + PlanState* outerNode, HashJoinState* hjstate, uint32* hashvalue) +{ + HashJoinTable hashtable = hjstate->hj_HashTable; + int curbatch = hashtable->curbatch; + TupleTableSlot* slot = NULL; + + /* + * In the Parallel Hash case we only run the outer plan directly for + * single-batch hash joins. Otherwise we have to go to batch files, even + * for batch 0. + */ + if (curbatch == 0 && hashtable->nbatch == 1) { + slot = ExecProcNode(outerNode); + + while (!TupIsNull(slot)) { + ExprContext* econtext = hjstate->js.ps.ps_ExprContext; + + econtext->ecxt_outertuple = slot; + if (ExecHashGetHashValue(hashtable, + econtext, + hjstate->hj_OuterHashKeys, + true, /* outer tuple */ + HJ_FILL_OUTER(hjstate) || hjstate->js.nulleqqual != NIL, + hashvalue)) { + return slot; + } + + /* + * That tuple couldn't match because of a NULL, so discard it and + * continue with the next one. + */ + slot = ExecProcNode(outerNode); + } + } else if (curbatch < hashtable->nbatch) { + MinimalTuple tuple; + + tuple = sts_parallel_scan_next(hashtable->batches[curbatch].outer_tuples, hashvalue); + if (tuple != NULL) { + slot = ExecStoreMinimalTuple(tuple, hjstate->hj_OuterTupleSlot, false); + return slot; + } else { + (void)ExecClearTuple(hjstate->hj_OuterTupleSlot); + } } /* End of this batch */ @@ -743,8 +976,9 @@ static bool ExecHashJoinNewBatch(HashJoinState* hjstate) * We no longer need the previous outer batch file; close it right * away to free disk space. */ - if (hashtable->outerBatchFile[curbatch]) + if (hashtable->outerBatchFile[curbatch]) { BufFileClose(hashtable->outerBatchFile[curbatch]); + } hashtable->outerBatchFile[curbatch] = NULL; /* we just finished the first batch */ } else { @@ -781,21 +1015,27 @@ static bool ExecHashJoinNewBatch(HashJoinState* hjstate) curbatch++; while (curbatch < nbatch && (hashtable->outerBatchFile[curbatch] == NULL || hashtable->innerBatchFile[curbatch] == NULL)) { - if (hashtable->outerBatchFile[curbatch] && HJ_FILL_OUTER(hjstate)) + if (hashtable->outerBatchFile[curbatch] && HJ_FILL_OUTER(hjstate)) { break; /* must process due to rule 1 */ - if (hashtable->innerBatchFile[curbatch] && HJ_FILL_INNER(hjstate)) + } + if (hashtable->innerBatchFile[curbatch] && HJ_FILL_INNER(hjstate)) { break; /* must process due to rule 1 */ - if (hashtable->innerBatchFile[curbatch] && nbatch != hashtable->nbatch_original) + } + if (hashtable->innerBatchFile[curbatch] && nbatch != hashtable->nbatch_original) { break; /* must process due to rule 2 */ - if (hashtable->outerBatchFile[curbatch] && nbatch != hashtable->nbatch_outstart) + } + if (hashtable->outerBatchFile[curbatch] && nbatch != hashtable->nbatch_outstart) { break; /* must process due to rule 3 */ + } /* We can ignore this batch. */ /* Release associated temp files right away. */ - if (hashtable->innerBatchFile[curbatch]) + if (hashtable->innerBatchFile[curbatch]) { BufFileClose(hashtable->innerBatchFile[curbatch]); + } hashtable->innerBatchFile[curbatch] = NULL; - if (hashtable->outerBatchFile[curbatch]) + if (hashtable->outerBatchFile[curbatch]) { BufFileClose(hashtable->outerBatchFile[curbatch]); + } hashtable->outerBatchFile[curbatch] = NULL; curbatch++; } @@ -832,8 +1072,9 @@ static bool ExecHashJoinNewBatch(HashJoinState* hjstate) } /* analysis hash table information created in memory */ - if (anls_opt_is_on(ANLS_HASH_CONFLICT)) + if (anls_opt_is_on(ANLS_HASH_CONFLICT)) { ExecHashTableStats(hashtable, hjstate->js.ps.plan->righttree->plan_node_id); + } /* * after we build the hash table, the inner batch file is no longer @@ -847,14 +1088,129 @@ static bool ExecHashJoinNewBatch(HashJoinState* hjstate) * Rewind outer batch file (if present), so that we can start reading it. */ if (hashtable->outerBatchFile[curbatch] != NULL) { - if (BufFileSeek(hashtable->outerBatchFile[curbatch], 0, 0L, SEEK_SET)) + if (BufFileSeek(hashtable->outerBatchFile[curbatch], 0, 0L, SEEK_SET)) { ereport( ERROR, (errcode_for_file_access(), errmsg("could not rewind hash-join probe side temporary file: %m"))); + } } return true; } +/* + * Choose a batch to work on, and attach to it. Returns true if successful, + * false if there are no more batches. + */ +static bool ExecParallelHashJoinNewBatch(HashJoinState* hjstate) +{ + HashJoinTable hashtable = hjstate->hj_HashTable; + int start_batchno; + int batchno; + + /* + * If we started up so late that the batch tracking array has been freed + * already by ExecHashTableDetach(), then we are finished. See also + * ExecParallelHashEnsureBatchAccessors(). + */ + if (hashtable->batches == NULL) { + return false; + } + + /* + * If we were already attached to a batch, remember not to bother checking + * it again, and detach from it (possibly freeing the hash table if we are + * last to detach). + */ + if (hashtable->curbatch >= 0) { + hashtable->batches[hashtable->curbatch].done = true; + ExecHashTableDetachBatch(hashtable); + } + + /* + * Search for a batch that isn't done. We use an atomic counter to start + * our search at a different batch in every participant when there are + * more batches than participants. + */ + batchno = start_batchno = ((int)pg_atomic_fetch_add_u32(&hashtable->parallel_state->distributor, 1)) % hashtable->nbatch; + do { + uint32 hashvalue; + MinimalTuple tuple; + TupleTableSlot* slot = NULL; + + if (!hashtable->batches[batchno].done) { + SharedTuplestoreAccessor* inner_tuples = NULL; + Barrier* batch_barrier = &hashtable->batches[batchno].shared->batch_barrier; + + switch (BarrierAttach(batch_barrier)) { + case PHJ_BATCH_ELECTING: + + /* One backend allocates the hash table. */ + if (BarrierArriveAndWait(batch_barrier, WAIT_EVENT_HASH_BATCH_ELECTING)) { + ExecParallelHashTableAlloc(hashtable, batchno); + } + /* Fall through. */ + + case PHJ_BATCH_ALLOCATING: + /* Wait for allocation to complete. */ + (void)BarrierArriveAndWait(batch_barrier, WAIT_EVENT_HASH_BATCH_ALLOCATING); + /* Fall through. */ + + case PHJ_BATCH_LOADING: + /* Start (or join in) loading tuples. */ + ExecParallelHashTableSetCurrentBatch(hashtable, batchno); + inner_tuples = hashtable->batches[batchno].inner_tuples; + sts_begin_parallel_scan(inner_tuples); + while ((tuple = sts_parallel_scan_next(inner_tuples, &hashvalue))) { + slot = ExecStoreMinimalTuple(tuple, hjstate->hj_HashTupleSlot, false); + ExecParallelHashTableInsertCurrentBatch(hashtable, slot, hashvalue); + } + sts_end_parallel_scan(inner_tuples); + (void)BarrierArriveAndWait(batch_barrier, WAIT_EVENT_HASH_BATCH_LOADING); + /* Fall through. */ + + case PHJ_BATCH_PROBING: + + /* + * This batch is ready to probe. Return control to + * caller. We stay attached to batch_barrier so that the + * hash table stays alive until everyone's finished + * probing it, but no participant is allowed to wait at + * this barrier again (or else a deadlock could occur). + * All attached participants must eventually call + * BarrierArriveAndDetach() so that the final phase + * PHJ_BATCH_DONE can be reached. + */ + ExecParallelHashTableSetCurrentBatch(hashtable, batchno); + sts_begin_parallel_scan(hashtable->batches[batchno].outer_tuples); + return true; + + case PHJ_BATCH_DONE: + + /* + * Already done. Detach and go around again (if any + * remain). + */ + (void)BarrierDetach(batch_barrier); + + /* + * We didn't work on this batch, but we need to observe + * its size for EXPLAIN. + */ + ExecParallelHashUpdateSpacePeak(hashtable, batchno); + hashtable->batches[batchno].done = true; + hashtable->curbatch = -1; + break; + + default: + elog(ERROR, "unexpected batch phase %d", BarrierPhase(batch_barrier)); + } + } + batchno = (batchno + 1) % hashtable->nbatch; + } while (batchno != start_batchno); + + return false; +} + /* * ExecHashJoinSaveTuple * save a tuple to a batch file. @@ -878,20 +1234,22 @@ void ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue, BufFile** filep } written = BufFileWrite(file, (void*)&hashvalue, sizeof(uint32)); - if (written != sizeof(uint32)) + if (written != sizeof(uint32)) { ereport(ERROR, (errcode_for_file_access(), errmsg("could not write hashvalue %u to hash-join temporary file, written length %lu.", hashvalue, written))); + } written = BufFileWrite(file, (void*)tuple, tuple->t_len); - if (written != tuple->t_len) + if (written != tuple->t_len) { ereport(ERROR, (errcode_for_file_access(), errmsg("could not write tuple to hash-join temporary file: written length %lu, tuple length %u", written, tuple->t_len))); + } /* increase current session spill count */ pgstat_increase_session_spill(); @@ -958,11 +1316,13 @@ void ExecReScanHashJoin(HashJoinState* node) { /* Already reset, just rescan righttree and lefttree */ if (node->js.ps.recursive_reset && node->js.ps.state->es_recursive_next_iteration) { - if (node->js.ps.righttree->chgParam == NULL) + if (node->js.ps.righttree->chgParam == NULL) { ExecReScan(node->js.ps.righttree); + } - if (node->js.ps.lefttree->chgParam == NULL) + if (node->js.ps.lefttree->chgParam == NULL) { ExecReScan(node->js.ps.lefttree); + } node->js.ps.recursive_reset = false; return; @@ -985,8 +1345,9 @@ void ExecReScanHashJoin(HashJoinState* node) * However, if it's a right/full join, we'd better reset the * inner-tuple match flags contained in the table. */ - if (HJ_FILL_INNER(node)) + if (HJ_FILL_INNER(node)) { ExecHashTableResetMatchFlags(node->hj_HashTable); + } /* * Also, we need to reset our state about the emptiness of the @@ -1012,8 +1373,9 @@ void ExecReScanHashJoin(HashJoinState* node) * by first ExecProcNode. */ // swtich to next partition, in the right tree - if (node->js.ps.righttree->chgParam == NULL) + if (node->js.ps.righttree->chgParam == NULL) { ExecReScan(node->js.ps.righttree); + } } } else { if (node->js.ps.plan->ispwj) { @@ -1042,8 +1404,9 @@ void ExecReScanHashJoin(HashJoinState* node) * if chgParam of subnode is not null then plan will be re-scanned by * first ExecProcNode. */ - if (node->js.ps.lefttree->chgParam == NULL) + if (node->js.ps.lefttree->chgParam == NULL) { ExecReScan(node->js.ps.lefttree); + } } /* @@ -1056,8 +1419,9 @@ void ExecEarlyFreeHashJoin(HashJoinState* node) { PlanState* plan_state = &node->js.ps; - if (plan_state->earlyFreed) + if (plan_state->earlyFreed) { return; + } /* * Free hash table @@ -1127,6 +1491,183 @@ void ExecReSetHashJoin(HashJoinState* node) * if chgParam of subnode is not null then plan will be re-scanned by * first ExecProcNode. */ - if (node->js.ps.lefttree->chgParam == NULL) + if (node->js.ps.lefttree->chgParam == NULL) { ExecReSetRecursivePlanTree(node->js.ps.lefttree); + } +} + +void ExecShutdownHashJoin(HashJoinState* node) +{ + if (node->hj_HashTable) { + /* + * Detach from shared state before DSM memory goes away. This makes + * sure that we don't have any pointers into DSM memory by the time + * ExecEndHashJoin runs. + */ + ExecHashTableDetachBatch(node->hj_HashTable); + ExecHashTableDetach(node->hj_HashTable); + } +} + +static void ExecParallelHashJoinPartitionOuter(HashJoinState* hjstate) +{ + PlanState* outerState = outerPlanState(hjstate); + ExprContext* econtext = hjstate->js.ps.ps_ExprContext; + HashJoinTable hashtable = hjstate->hj_HashTable; + TupleTableSlot* slot = NULL; + uint32 hashvalue; + int i; + + Assert(hjstate->hj_FirstOuterTupleSlot == NULL); + + /* Execute outer plan, writing all tuples to shared tuplestores. */ + for (;;) { + slot = ExecProcNode(outerState); + if (TupIsNull(slot)) { + break; + } + econtext->ecxt_outertuple = slot; + if (ExecHashGetHashValue(hashtable, + econtext, + hjstate->hj_OuterHashKeys, + true, /* outer tuple */ + false, /* outer join, currently unsupported */ + &hashvalue)) { + int batchno; + int bucketno; + + ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno); + sts_puttuple(hashtable->batches[batchno].outer_tuples, &hashvalue, ExecFetchSlotMinimalTuple(slot)); + } + CHECK_FOR_INTERRUPTS(); + } + + /* Make sure all outer partitions are readable by any backend. */ + for (i = 0; i < hashtable->nbatch; ++i) { + sts_end_write(hashtable->batches[i].outer_tuples); + } +} + +void ExecHashJoinInitializeDSM(HashJoinState* state, ParallelContext* pcxt, int nodeid) +{ + int plan_node_id = state->js.ps.plan->plan_node_id; + HashState* hashNode = NULL; + ParallelHashJoinState* pstate = NULL; + + /* + * Disable shared hash table mode if we failed to create a real DSM + * segment, because that means that we don't have a DSA area to work with. + */ + if (pcxt->seg == NULL) { + return; + } + state->isParallel = true; + + /* + * Set up the state needed to coordinate access to the shared hash + * table(s), using the plan node ID as the toc key. + */ + knl_u_parallel_context* cxt = (knl_u_parallel_context*)pcxt->seg; + pstate = (ParallelHashJoinState*)MemoryContextAllocZero(cxt->memCtx, sizeof(ParallelHashJoinState)); + pstate->plan_node_id = plan_node_id; + cxt->pwCtx->queryInfo.jstate[nodeid] = pstate; + /* + * Set up the shared hash join state with no batches initially. + * ExecHashTableCreate() will prepare at least one later and set nbatch + * and space_allowed. + */ + pstate->nbatch = 0; + pstate->space_allowed = 0; + pstate->batches = InvalidDsaPointer; + pstate->old_batches = InvalidDsaPointer; + pstate->nbuckets = 0; + pstate->growth = PHJ_GROWTH_OK; + pstate->chunk_work_queue = InvalidDsaPointer; + pg_atomic_init_u32(&pstate->distributor, 0); + pstate->nparticipants = pcxt->nworkers + 1; + pstate->total_tuples = 0; + LWLockInitialize(&pstate->lock, LWTRANCHE_PARALLEL_HASH_JOIN); + BarrierInit(&pstate->build_barrier, 0); + BarrierInit(&pstate->grow_batches_barrier, 0); + BarrierInit(&pstate->grow_buckets_barrier, 0); + + /* Set up the space we'll use for shared temporary files. */ + SharedFileSetInit(&pstate->fileset, pcxt->seg); + + /* Initialize the shared state in the hash node. */ + hashNode = (HashState*)innerPlanState(state); + hashNode->parallel_state = pstate; + t_thrd.bgworker_cxt.memCxt = cxt->memCtx; +} + +/* ---------------------------------------------------------------- + * ExecHashJoinReInitializeDSM + * + * Reset shared state before beginning a fresh scan. + * ---------------------------------------------------------------- + */ +void ExecHashJoinReInitializeDSM(HashJoinState* state, ParallelContext* parallelCxt) +{ + int planNodeId = state->js.ps.plan->plan_node_id; + ParallelHashJoinState* pstate = NULL; + knl_u_parallel_context *cxt = (knl_u_parallel_context *)parallelCxt->seg; + for (int i = 0; i < cxt->pwCtx->queryInfo.jstate_num; i++) { + if (planNodeId == ((ParallelHashJoinState*)cxt->pwCtx->queryInfo.jstate[i])->plan_node_id) { + pstate = (ParallelHashJoinState*)cxt->pwCtx->queryInfo.jstate[i]; + break; + } + } + + if (pstate == NULL) { + ereport(ERROR, (errmsg("could not find plan info, plan node id:%d", planNodeId))); + } + /* + * It would be possible to reuse the shared hash table in single-batch + * cases by resetting and then fast-forwarding build_barrier to + * PHJ_BUILD_DONE and batch 0's batch_barrier to PHJ_BATCH_PROBING, but + * currently shared hash tables are already freed by now (by the last + * participant to detach from the batch). We could consider keeping it + * around for single-batch joins. We'd also need to adjust + * finalize_plan() so that it doesn't record a dummy dependency for + * Parallel Hash nodes, preventing the rescan optimization. For now we + * don't try. + */ + + /* Detach, freeing any remaining shared memory. */ + if (state->hj_HashTable != NULL) { + ExecHashTableDetachBatch(state->hj_HashTable); + ExecHashTableDetach(state->hj_HashTable); + } + + /* Clear any shared batch files. */ + SharedFileSetDeleteAll(&pstate->fileset); + + /* Reset build_barrier to PHJ_BUILD_ELECTING so we can go around again. */ + BarrierInit(&pstate->build_barrier, 0); +} + +void ExecHashJoinInitializeWorker(HashJoinState* state, void* pwcxt) +{ + knl_u_parallel_context* cxt = (knl_u_parallel_context*)pwcxt; + HashState* hashNode = NULL; + int plan_node_id = state->js.ps.plan->plan_node_id; + + ParallelHashJoinState* pstate = NULL; + + for (int i = 0; i < cxt->pwCtx->queryInfo.jstate_num; i++) { + if (plan_node_id == ((ParallelHashJoinState*)cxt->pwCtx->queryInfo.jstate[i])->plan_node_id) { + pstate = (ParallelHashJoinState*)cxt->pwCtx->queryInfo.jstate[i]; + break; + } + } + + if (pstate == NULL) { + ereport(ERROR, (errmsg("could not find plan info, plan node id:%d", plan_node_id))); + } + + /* Attach to the shared state in the hash node. */ + hashNode = (HashState*)innerPlanState(state); + hashNode->parallel_state = pstate; + state->isParallel = true; + t_thrd.bgworker_cxt.memCxt = cxt->memCtx; } diff --git a/src/gausskernel/runtime/executor/nodeIndexonlyscan.cpp b/src/gausskernel/runtime/executor/nodeIndexonlyscan.cpp index 421c3d6ab..e099dde2a 100755 --- a/src/gausskernel/runtime/executor/nodeIndexonlyscan.cpp +++ b/src/gausskernel/runtime/executor/nodeIndexonlyscan.cpp @@ -588,21 +588,24 @@ IndexOnlyScanState* ExecInitIndexOnlyScan(IndexOnlyScan* node, EState* estate, i /* Initialize table partition list and index partition list for following scan*/ ExecInitPartitionForIndexOnlyScan(indexstate, estate); - /* construct a dummy table relation with the first table partition for following scan */ - currentpartition = (Partition)list_nth(indexstate->ss.partitions, 0); - indexstate->ss.ss_currentPartition = - partitionGetRelation(indexstate->ss.ss_currentRelation, currentpartition); + if (indexstate->ss.partitions != NIL) { + /* construct a dummy table relation with the first table partition for following scan */ + currentpartition = (Partition)list_nth(indexstate->ss.partitions, 0); + indexstate->ss.ss_currentPartition = + partitionGetRelation(indexstate->ss.ss_currentRelation, currentpartition); - /* construct a dummy index relation with the first table partition for following scan */ - currentindex = (Partition)list_nth(indexstate->ioss_IndexPartitionList, 0); - indexstate->ioss_CurrentIndexPartition = partitionGetRelation(indexstate->ioss_RelationDesc, currentindex); + /* construct a dummy index relation with the first table partition for following scan */ + currentindex = (Partition)list_nth(indexstate->ioss_IndexPartitionList, 0); + indexstate->ioss_CurrentIndexPartition = partitionGetRelation(indexstate->ioss_RelationDesc, + currentindex); - indexstate->ioss_ScanDesc = abs_idx_beginscan(indexstate->ss.ss_currentPartition, - indexstate->ioss_CurrentIndexPartition, - estate->es_snapshot, - indexstate->ioss_NumScanKeys, - indexstate->ioss_NumOrderByKeys, - (ScanState*)indexstate); + indexstate->ioss_ScanDesc = abs_idx_beginscan(indexstate->ss.ss_currentPartition, + indexstate->ioss_CurrentIndexPartition, + estate->es_snapshot, + indexstate->ioss_NumScanKeys, + indexstate->ioss_NumOrderByKeys, + (ScanState*)indexstate); + } } } else { /* @@ -743,7 +746,6 @@ void ExecInitPartitionForIndexOnlyScan(IndexOnlyScanState* indexstate, EState* e ListCell* cell = NULL; List* part_seqs = plan->scan.pruningInfo->ls_rangeSelectedPartitions; - Assert(plan->scan.itrs == plan->scan.pruningInfo->ls_rangeSelectedPartitions->length); /* * get relation's lockmode that hangs on whether * it's one of the target relations of the query @@ -753,6 +755,38 @@ void ExecInitPartitionForIndexOnlyScan(IndexOnlyScanState* indexstate, EState* e indexstate->ss.lockMode = lock; indexstate->lockMode = lock; + if (plan->scan.pruningInfo->paramArg != NULL) { + Param *paramArg = plan->scan.pruningInfo->paramArg; + Oid tablepartitionid = getPartitionOidByParam(currentRelation, paramArg, + &(estate->es_param_list_info->params[paramArg->paramid - 1])); + if (OidIsValid(tablepartitionid)) { + List *partitionIndexOidList = NIL; + tablepartition = partitionOpen(currentRelation, tablepartitionid, lock); + indexstate->ss.partitions = lappend(indexstate->ss.partitions, tablepartition); + partitionIndexOidList = PartitionGetPartIndexList(tablepartition); + + Assert(PointerIsValid(partitionIndexOidList)); + if (!PointerIsValid(partitionIndexOidList)) { + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmsg("no local indexes found for partition %s", + PartitionGetPartitionName(tablepartition)))); + } + + Oid indexpartitionid = searchPartitionIndexOid(indexid, partitionIndexOidList); + list_free(partitionIndexOidList); + indexpartition = partitionOpen(indexstate->ioss_RelationDesc, indexpartitionid, lock); + if (indexpartition->pd_part->indisusable == false) { + elog(ERROR, "can'nt initialize index-only scans using unusable local index \"%s\"", + PartitionGetPartitionName(indexpartition)); + } + + indexstate->ioss_IndexPartitionList = lappend(indexstate->ioss_IndexPartitionList, indexpartition); + } + return; + } + + Assert(plan->scan.itrs == plan->scan.pruningInfo->ls_rangeSelectedPartitions->length); foreach (cell, part_seqs) { Oid indexpartitionid = InvalidOid; List* partitionIndexOidList = NIL; diff --git a/src/gausskernel/runtime/executor/nodeIndexscan.cpp b/src/gausskernel/runtime/executor/nodeIndexscan.cpp index 7234f4aae..ce5fc4c28 100755 --- a/src/gausskernel/runtime/executor/nodeIndexscan.cpp +++ b/src/gausskernel/runtime/executor/nodeIndexscan.cpp @@ -23,6 +23,7 @@ * ExecIndexRestrPos restores scan position. * ExecIndexScanEstimate estimates DSM space needed for parallel index scan * ExecIndexScanInitializeDSM initialize DSM for parallel indexscan + * ExecIndexScanReInitializeDSM reinitialize DSM for fresh scan * ExecIndexScanInitializeWorker attach to DSM info in parallel worker */ #include "postgres.h" @@ -182,19 +183,6 @@ TupleTableSlot* ExecIndexScan(IndexScanState* node) */ void ExecReScanIndexScan(IndexScanState* node) { - bool reset_parallel_scan = true; - - /* - * If we are here to just update the scan keys, then don't reset parallel - * scan. We don't want each of the participating process in the parallel - * scan to update the shared parallel scan state at the start of the scan. - * It is quite possible that one of the participants has already begun - * scanning the index when another has yet to start it. - */ - if (node->iss_NumRuntimeKeys != 0 && !node->iss_RuntimeKeysReady) { - reset_parallel_scan = false; - } - /* * For recursive-stream rescan, if number of RuntimeKeys not euqal zero, * just return without rescan. @@ -256,10 +244,6 @@ void ExecReScanIndexScan(IndexScanState* node) if (node->iss_ScanDesc) { abs_idx_rescan(node->iss_ScanDesc, node->iss_ScanKeys, node->iss_NumScanKeys, node->iss_OrderByKeys, node->iss_NumOrderByKeys); - - if (reset_parallel_scan && GetIndexScanDesc(node->iss_ScanDesc)->parallel_scan) { - index_parallelrescan(GetIndexScanDesc(node->iss_ScanDesc)); - } } ExecScanReScan(&node->ss); @@ -685,23 +669,26 @@ IndexScanState* ExecInitIndexScan(IndexScan* node, EState* estate, int eflags) /* Initialize table partition list and index partition list for following scan */ ExecInitPartitionForIndexScan(index_state, estate); - /* construct a dummy relation with the first table partition for following scan */ - current_partition = (Partition)list_nth(index_state->ss.partitions, 0); - index_state->ss.ss_currentPartition = - partitionGetRelation(index_state->ss.ss_currentRelation, current_partition); + if (index_state->ss.partitions != NIL) { + /* construct a dummy relation with the first table partition for following scan */ + current_partition = (Partition)list_nth(index_state->ss.partitions, 0); + index_state->ss.ss_currentPartition = + partitionGetRelation(index_state->ss.ss_currentRelation, current_partition); - /* construct a dummy relation with the first table partition for following scan */ - currentindex = (Partition)list_nth(index_state->iss_IndexPartitionList, 0); - index_state->iss_CurrentIndexPartition = partitionGetRelation(index_state->iss_RelationDesc, currentindex); + /* construct a dummy relation with the first table partition for following scan */ + currentindex = (Partition)list_nth(index_state->iss_IndexPartitionList, 0); + index_state->iss_CurrentIndexPartition = partitionGetRelation(index_state->iss_RelationDesc, + currentindex); - /* Initialize scan descriptor for partitioned table */ - index_state->iss_ScanDesc = abs_idx_beginscan(index_state->ss.ss_currentPartition, - index_state->iss_CurrentIndexPartition, - estate->es_snapshot, - index_state->iss_NumScanKeys, - index_state->iss_NumOrderByKeys, - (ScanState*)index_state); - Assert(PointerIsValid(index_state->iss_ScanDesc)); + /* Initialize scan descriptor for partitioned table */ + index_state->iss_ScanDesc = abs_idx_beginscan(index_state->ss.ss_currentPartition, + index_state->iss_CurrentIndexPartition, + estate->es_snapshot, + index_state->iss_NumScanKeys, + index_state->iss_NumOrderByKeys, + (ScanState*)index_state); + Assert(PointerIsValid(index_state->iss_ScanDesc)); + } } } else { /* @@ -1337,6 +1324,36 @@ void ExecInitPartitionForIndexScan(IndexScanState* index_state, EState* estate) index_state->ss.lockMode = lock; index_state->lockMode = lock; + if (plan->scan.pruningInfo->paramArg != NULL) { + Param *paramArg = plan->scan.pruningInfo->paramArg; + Oid tablepartitionid = getPartitionOidByParam(current_relation, paramArg, + &(estate->es_param_list_info->params[paramArg->paramid - 1])); + if (OidIsValid(tablepartitionid)) { + List *partitionIndexOidList = NIL; + table_partition = partitionOpen(current_relation, tablepartitionid, lock); + index_state->ss.partitions = lappend(index_state->ss.partitions, table_partition); + partitionIndexOidList = PartitionGetPartIndexList(table_partition); + + Assert(PointerIsValid(partitionIndexOidList)); + if (!PointerIsValid(partitionIndexOidList)) { + ereport(ERROR, + (errcode(ERRCODE_WRONG_OBJECT_TYPE), + errmsg("no local indexes found for partition %s", PartitionGetPartitionName(table_partition)))); + } + + Oid indexpartitionid = searchPartitionIndexOid(indexid, partitionIndexOidList); + list_free(partitionIndexOidList); + index_partition = partitionOpen(index_state->iss_RelationDesc, indexpartitionid, lock); + if (index_partition->pd_part->indisusable == false) { + elog(ERROR, "can'nt initialize index scans using unusable local index \"%s\"", + PartitionGetPartitionName(index_partition)); + } + + index_state->iss_IndexPartitionList = lappend(index_state->iss_IndexPartitionList, index_partition); + } + return; + } + Assert(plan->scan.itrs == plan->scan.pruningInfo->ls_rangeSelectedPartitions->length); foreach (cell, part_seqs) { @@ -1420,6 +1437,17 @@ void ExecIndexScanInitializeDSM(IndexScanState *node, ParallelContext *pcxt, int } } +/* ---------------------------------------------------------------- + * ExecIndexScanReInitializeDSM + * + * Reset shared state before beginning a fresh scan. + * ---------------------------------------------------------------- + */ +void ExecIndexScanReInitializeDSM(IndexScanState* node, ParallelContext* pcxt) +{ + index_parallelrescan((IndexScanDesc)node->iss_ScanDesc); +} + /* ---------------------------------------------------------------- * ExecIndexScanInitializeWorker * diff --git a/src/gausskernel/runtime/executor/nodeSeqscan.cpp b/src/gausskernel/runtime/executor/nodeSeqscan.cpp index 8557f8a23..11e410910 100755 --- a/src/gausskernel/runtime/executor/nodeSeqscan.cpp +++ b/src/gausskernel/runtime/executor/nodeSeqscan.cpp @@ -393,25 +393,37 @@ void InitScanRelation(SeqScanState* node, EState* estate) } node->lockMode = lockmode; + /* Generate node->partitions if exists */ if (plan->itrs > 0) { Partition part = NULL; - Partition currentPart = NULL; - ListCell* cell = NULL; - List* part_seqs = plan->pruningInfo->ls_rangeSelectedPartitions; + if (plan->pruningInfo->paramArg != NULL) { + Param *paramArg = plan->pruningInfo->paramArg; + Oid tablepartitionid = getPartitionOidByParam(current_relation, paramArg, + &(estate->es_param_list_info->params[paramArg->paramid - 1])); + if (OidIsValid(tablepartitionid)) { + part = partitionOpen(current_relation, tablepartitionid, lockmode); + node->partitions = lappend(node->partitions, part); + } + } else { + ListCell* cell = NULL; + List* part_seqs = plan->pruningInfo->ls_rangeSelectedPartitions; - Assert(plan->itrs == part_seqs->length); + Assert(plan->itrs == part_seqs->length); - foreach (cell, part_seqs) { - Oid tablepartitionid = InvalidOid; - int partSeq = lfirst_int(cell); + foreach (cell, part_seqs) { + Oid tablepartitionid = InvalidOid; + int partSeq = lfirst_int(cell); - tablepartitionid = getPartitionOidFromSequence(current_relation, partSeq); - part = partitionOpen(current_relation, tablepartitionid, lockmode); - node->partitions = lappend(node->partitions, part); + tablepartitionid = getPartitionOidFromSequence(current_relation, partSeq); + part = partitionOpen(current_relation, tablepartitionid, lockmode); + node->partitions = lappend(node->partitions, part); + } } + } + if (node->partitions != NIL) { /* construct HeapScanDesc for first partition */ - currentPart = (Partition)list_nth(node->partitions, 0); + Partition currentPart = (Partition)list_nth(node->partitions, 0); current_part_rel = partitionGetRelation(current_relation, currentPart); node->ss_currentPartition = current_part_rel; @@ -680,6 +692,18 @@ void ExecSeqScanInitializeDSM(SeqScanState *node, ParallelContext *pcxt, int nod (AbsTblScanDesc)heap_beginscan_parallel(node->ss_currentRelation, cxt->pwCtx->queryInfo.pscan[nodeid]); } +/* ---------------------------------------------------------------- + * ExecSeqScanReInitializeDSM + * + * Reset shared state before beginning a fresh scan. + * ---------------------------------------------------------------- + */ +void ExecSeqScanReInitializeDSM(SeqScanState* node, ParallelContext* pcxt) +{ + HeapScanDesc scan = (HeapScanDesc)node->ss_currentScanDesc; + heap_parallelscan_reinitialize(scan->rs_parallel); +} + /* ---------------------------------------------------------------- * ExecSeqScanInitializeWorker * diff --git a/src/gausskernel/runtime/executor/opfusion.cpp b/src/gausskernel/runtime/executor/opfusion.cpp index 84e71175a..effcf2a7f 100644 --- a/src/gausskernel/runtime/executor/opfusion.cpp +++ b/src/gausskernel/runtime/executor/opfusion.cpp @@ -30,6 +30,7 @@ #include "access/transam.h" #include "catalog/pg_aggregate.h" #include "catalog/storage_gtt.h" +#include "catalog/heap.h" #include "commands/copy.h" #include "executor/nodeIndexscan.h" #include "gstrace/executer_gstrace.h" @@ -804,7 +805,11 @@ SelectFusion::SelectFusion(MemoryContext context, CachedPlanSource* psrc, List* /* get limit num */ if (IsA(m_planstmt->planTree, Limit)) { Limit* limit = (Limit*)m_planstmt->planTree; - node = (Node*)m_planstmt->planTree->lefttree; + if (IsA(m_planstmt->planTree->lefttree, PartIterator)) { + node = (Node*)m_planstmt->planTree->lefttree->lefttree; + } else { + node = (Node*)m_planstmt->planTree->lefttree; + } if (limit->limitCount != NULL && IsA(limit->limitCount, Const) && !((Const*)limit->limitCount)->constisnull) { m_limitCount = DatumGetInt64(((Const*)limit->limitCount)->constvalue); } @@ -813,7 +818,11 @@ SelectFusion::SelectFusion(MemoryContext context, CachedPlanSource* psrc, List* m_limitOffset = DatumGetInt64(((Const*)limit->limitOffset)->constvalue); } } else { - node = (Node*)m_planstmt->planTree; + if (IsA(m_planstmt->planTree, PartIterator)) { + node = (Node*)m_planstmt->planTree->lefttree; + } else { + node = (Node*)m_planstmt->planTree; + } } initParams(params); @@ -1072,6 +1081,9 @@ void InsertFusion::refreshParameterIfNecessary() bool InsertFusion::execute(long max_rows, char* completionTag) { bool success = false; + Oid partOid = InvalidOid; + Partition part = NULL; + Relation partRel = NULL; /******************* * step 1: prepare * @@ -1097,10 +1109,15 @@ bool InsertFusion::execute(long max_rows, char* completionTag) ************************/ HeapTuple tuple = heap_form_tuple(m_tupDesc, m_values, m_isnull); Assert(tuple != NULL); + if (RELATION_IS_PARTITIONED(rel)) { + partOid = heapTupleGetPartitionId(rel, tuple); + part = partitionOpen(rel, partOid, RowExclusiveLock); + partRel = partitionGetRelation(rel, part); + } if (m_is_bucket_rel) { bucketid = computeTupleBucketId(result_rel_info->ri_RelationDesc, tuple); if (bucketid != InvalidBktId) { - bucket_rel = bucketGetRelation(rel, NULL, bucketid); + bucket_rel = bucketGetRelation(rel, part, bucketid); } else { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), @@ -1114,12 +1131,37 @@ bool InsertFusion::execute(long max_rows, char* completionTag) ExecConstraints(result_rel_info, m_reslot, m_estate); } - (void)heap_insert(bucket_rel == NULL ? rel : bucket_rel, tuple, mycid, 0, NULL); + switch (result_rel_info->ri_RelationDesc->rd_rel->parttype) { + case PARTTYPE_NON_PARTITIONED_RELATION: + case PARTTYPE_VALUE_PARTITIONED_RELATION: { + (void)heap_insert(bucket_rel == NULL ? rel : bucket_rel, tuple, mycid, 0, NULL); + } break; + + case PARTTYPE_PARTITIONED_RELATION: { + (void)heap_insert(bucket_rel == NULL ? partRel : bucket_rel, tuple, mycid, 0, NULL); + } break; + + default: { + /* never happen; just to be self-contained */ + ereport(ERROR, + (errmodule(MOD_EXECUTOR), + (errcode(ERRCODE_UNRECOGNIZED_NODE_TYPE), + errmsg("Unrecognized parttype as \"%c\" for relation \"%s\"", + RelationGetPartType(result_rel_info->ri_RelationDesc), + RelationGetRelationName(result_rel_info->ri_RelationDesc))))); + } break; + } /* insert index entries for tuple */ List* recheck_indexes = NIL; if (result_rel_info->ri_NumIndices > 0) { - recheck_indexes = ExecInsertIndexTuples(m_reslot, &(tuple->t_self), m_estate, NULL, NULL, bucketid, NULL); + recheck_indexes = ExecInsertIndexTuples(m_reslot, + &(tuple->t_self), + m_estate, + RELATION_IS_PARTITIONED(rel) ? partRel : NULL, + RELATION_IS_PARTITIONED(rel) ? part : NULL, + bucketid, + NULL); } list_free_ext(recheck_indexes); @@ -1143,6 +1185,11 @@ bool InsertFusion::execute(long max_rows, char* completionTag) bucketCloseRelation(bucket_rel); } + if (RELATION_IS_PARTITIONED(rel)) { + partitionClose(rel, part, RowExclusiveLock); + releaseDummyRelation(&partRel); + } + errno_t errorno = snprintf_s(completionTag, COMPLETION_TAG_BUFSIZE, COMPLETION_TAG_BUFSIZE - 1, "INSERT 0 1"); securec_check_ss(errorno, "\0", "\0"); @@ -1255,6 +1302,10 @@ UpdateFusion::UpdateFusion(MemoryContext context, CachedPlanSource* psrc, List* MemoryContext old_context = MemoryContextSwitchTo(m_context); ModifyTable* node = (ModifyTable*)m_planstmt->planTree; + Plan *updatePlan = (Plan *)linitial(node->plans); + if (IsA(updatePlan, PartIterator)) { + updatePlan = updatePlan->lefttree; + } m_reloid = getrelid(linitial_int(m_planstmt->resultRelations), m_planstmt->rtable); Relation rel = heap_open(m_reloid, AccessShareLock); @@ -1270,13 +1321,17 @@ UpdateFusion::UpdateFusion(MemoryContext context, CachedPlanSource* psrc, List* heap_close(rel, AccessShareLock); - IndexScan* indexscan = (IndexScan*)linitial(node->plans); + IndexScan* indexscan = (IndexScan*)updatePlan; if (m_is_bucket_rel) { // ctid + tablebucketid Assert(RelationGetDescr(rel)->natts + 2 == list_length(indexscan->scan.plan.targetlist)); } else { // ctid - Assert(RelationGetDescr(rel)->natts + 1 == list_length(indexscan->scan.plan.targetlist)); + if (indexscan->scan.isPartTbl) { + Assert(RelationGetDescr(rel)->natts + 2 == list_length(indexscan->scan.plan.targetlist)); + } else { + Assert(RelationGetDescr(rel)->natts + 1 == list_length(indexscan->scan.plan.targetlist)); + } } m_outParams = params; @@ -1343,7 +1398,7 @@ UpdateFusion::UpdateFusion(MemoryContext context, CachedPlanSource* psrc, List* m_receiver = NULL; m_isInsideRec = true; - m_scan = ScanFusion::getScanFusion((Node*)linitial(node->plans), m_planstmt, m_outParams); + m_scan = ScanFusion::getScanFusion((Node*)indexscan, m_planstmt, m_outParams); MemoryContextSwitchTo(old_context); } @@ -1380,15 +1435,17 @@ void UpdateFusion::refreshTargetParameterIfNecessary() bool UpdateFusion::execute(long max_rows, char* completionTag) { bool success = false; + IndexScanFusion* scan = (IndexScanFusion*)m_scan; /******************* * step 1: prepare * *******************/ m_scan->refreshParameter(m_outParams == NULL ? m_params : m_outParams); - m_scan->Init(max_rows); - Relation rel = m_scan->m_rel; + Relation rel = (m_scan->m_parentRel == NULL ? m_scan->m_rel : m_scan->m_parentRel); + Relation partRel = NULL; + Partition part = NULL; Relation bucket_rel = NULL; int2 bucketid = InvalidBktId; @@ -1400,7 +1457,10 @@ bool UpdateFusion::execute(long max_rows, char* completionTag) if (result_rel_info->ri_RelationDesc->rd_rel->relhasindex) { ExecOpenIndices(result_rel_info, false); } - + if (RELATION_IS_PARTITIONED(rel)) { + partRel = scan->m_rel; + part = scan->m_partRel; + } /********************************* * step 2: begin scan and update * *********************************/ @@ -1412,7 +1472,7 @@ bool UpdateFusion::execute(long max_rows, char* completionTag) while ((oldtup = m_scan->getTuple()) != NULL) { - if (RelationIsPartitioned(m_scan->m_rel)) { + if (RELATION_IS_PARTITIONED(m_scan->m_rel)) { rel = m_scan->getCurrentRel(); } @@ -1427,7 +1487,7 @@ bool UpdateFusion::execute(long max_rows, char* completionTag) Assert(tup->t_bucketId == computeTupleBucketId(result_rel_info->ri_RelationDesc, tup)); bucketid = tup->t_bucketId; if (bucketid != InvalidBktId) { - bucket_rel = bucketGetRelation(rel, NULL, bucketid); + bucket_rel = bucketGetRelation(rel, part, bucketid); } else { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), @@ -1439,15 +1499,28 @@ bool UpdateFusion::execute(long max_rows, char* completionTag) if (rel->rd_att->constr) ExecConstraints(result_rel_info, m_reslot, m_estate); - result = heap_update(bucket_rel == NULL ? rel : bucket_rel, - bucket_rel == NULL ? NULL : rel, - &oldtup->t_self, - tup, - &update_ctid, - &update_xmax, - m_estate->es_output_cid, - InvalidSnapshot, - true); + if (RELATION_IS_PARTITIONED(rel)) { + result = heap_update(bucket_rel == NULL ? partRel : bucket_rel, + rel, + &oldtup->t_self, + tup, + &update_ctid, + &update_xmax, + m_estate->es_output_cid, + InvalidSnapshot, + true); + } else { + result = heap_update(bucket_rel == NULL ? rel : bucket_rel, + bucket_rel == NULL ? NULL : rel, + &oldtup->t_self, + tup, + &update_ctid, + &update_xmax, + m_estate->es_output_cid, + InvalidSnapshot, + true); + } + switch (result) { case HeapTupleSelfUpdated: /* already deleted by self; nothing to do */ @@ -1458,7 +1531,7 @@ bool UpdateFusion::execute(long max_rows, char* completionTag) nprocessed++; if (result_rel_info->ri_NumIndices > 0 && !HeapTupleIsHeapOnly(tup)) { recheck_indexes = ExecInsertIndexTuples(m_reslot, &(tup->t_self), m_estate, - NULL, NULL, bucketid, NULL); + partRel, part, bucketid, NULL); list_free_ext(recheck_indexes); } break; @@ -1473,11 +1546,19 @@ bool UpdateFusion::execute(long max_rows, char* completionTag) bool* isnullfornew = NULL; Datum* valuesfornew = NULL; HeapTuple copyTuple; - copyTuple = EvalPlanQualFetch(m_estate, + if (RELATION_IS_PARTITIONED(rel)) { + copyTuple = EvalPlanQualFetch(m_estate, + bucket_rel == NULL ? partRel : bucket_rel, + LockTupleExclusive, + &update_ctid, + update_xmax); + } else { + copyTuple = EvalPlanQualFetch(m_estate, bucket_rel == NULL ? rel : bucket_rel, LockTupleExclusive, &update_ctid, update_xmax); + } if (copyTuple == NULL) { pfree_ext(valuesfornew); pfree_ext(isnullfornew); @@ -1500,6 +1581,18 @@ bool UpdateFusion::execute(long max_rows, char* completionTag) pfree(isnullfornew); goto lreplace; + } else if (result_rel_info->ri_RelationDesc->rd_rel->relrowmovement) { + if (RELATION_IS_PARTITIONED(rel)) { + /* + * the may be a row movement update action which delete tuple from original + * partition and insert tuple to new partition or we can add lock on the tuple to + * be delete or updated to avoid throw exception + */ + ereport(ERROR, + (errcode(ERRCODE_TRANSACTION_ROLLBACK), + errmsg("partition table update conflict"), + errdetail("disable row movement of table can avoid this conflict"))); + } } break; @@ -1543,7 +1636,11 @@ DeleteFusion::DeleteFusion(MemoryContext context, CachedPlanSource* psrc, List* MemoryContext old_context = MemoryContextSwitchTo(m_context); ModifyTable* node = (ModifyTable*)m_planstmt->planTree; - + Plan *deletePlan = (Plan *)linitial(node->plans); + if (IsA(deletePlan, PartIterator)) { + deletePlan = deletePlan->lefttree; + } + IndexScan* indexscan = (IndexScan*)deletePlan; m_reloid = getrelid(linitial_int(m_planstmt->resultRelations), m_planstmt->rtable); Relation rel = heap_open(m_reloid, AccessShareLock); @@ -1563,7 +1660,7 @@ DeleteFusion::DeleteFusion(MemoryContext context, CachedPlanSource* psrc, List* m_receiver = NULL; m_isInsideRec = true; - m_scan = ScanFusion::getScanFusion((Node*)linitial(node->plans), m_planstmt, m_outParams); + m_scan = ScanFusion::getScanFusion((Node*)indexscan, m_planstmt, m_outParams); MemoryContextSwitchTo(old_context); } @@ -1571,6 +1668,7 @@ DeleteFusion::DeleteFusion(MemoryContext context, CachedPlanSource* psrc, List* bool DeleteFusion::execute(long max_rows, char* completionTag) { bool success = false; + IndexScanFusion* scan = (IndexScanFusion*)m_scan; /******************* * step 1: prepare * @@ -1579,7 +1677,9 @@ bool DeleteFusion::execute(long max_rows, char* completionTag) m_scan->Init(max_rows); - Relation rel = m_scan->m_rel; + Relation rel = (m_scan->m_parentRel == NULL ? m_scan->m_rel : m_scan->m_parentRel); + Relation partRel = NULL; + Partition part = NULL; Relation bucket_rel = NULL; int2 bucketid = InvalidBktId; @@ -1591,7 +1691,10 @@ bool DeleteFusion::execute(long max_rows, char* completionTag) if (result_rel_info->ri_RelationDesc->rd_rel->relhasindex) { ExecOpenIndices(result_rel_info, false); } - + if (RELATION_IS_PARTITIONED(rel)) { + partRel = scan->m_rel; + part = scan->m_partRel; + } /******************************** * step 2: begin scan and delete* ********************************/ @@ -1611,7 +1714,7 @@ bool DeleteFusion::execute(long max_rows, char* completionTag) Assert(oldtup->t_bucketId == computeTupleBucketId(result_rel_info->ri_RelationDesc, oldtup)); bucketid = oldtup->t_bucketId; if (bucketid != InvalidBktId) { - bucket_rel = bucketGetRelation(rel, NULL, bucketid); + bucket_rel = bucketGetRelation(rel, part, bucketid); } else { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), @@ -1620,13 +1723,23 @@ bool DeleteFusion::execute(long max_rows, char* completionTag) } ldelete: - result = heap_delete(bucket_rel == NULL ? rel : bucket_rel, + if (RELATION_IS_PARTITIONED(rel)) { + result = heap_delete(bucket_rel == NULL ? partRel : bucket_rel, &oldtup->t_self, &update_ctid, &update_xmax, m_estate->es_output_cid, InvalidSnapshot, true); + } else { + result = heap_delete(bucket_rel == NULL ? rel : bucket_rel, + &oldtup->t_self, + &update_ctid, + &update_xmax, + m_estate->es_output_cid, + InvalidSnapshot, + true); + } switch (result) { case HeapTupleSelfUpdated: /* already deleted by self; nothing to do */ @@ -1647,12 +1760,19 @@ bool DeleteFusion::execute(long max_rows, char* completionTag) bool* isnullfornew = NULL; Datum* valuesfornew = NULL; HeapTuple copyTuple; - - copyTuple = EvalPlanQualFetch(m_estate, + if (RELATION_IS_PARTITIONED(rel)) { + copyTuple = EvalPlanQualFetch(m_estate, + bucket_rel == NULL ? partRel : bucket_rel, + LockTupleExclusive, + &update_ctid, + update_xmax); + } else { + copyTuple = EvalPlanQualFetch(m_estate, bucket_rel == NULL ? rel : bucket_rel, LockTupleExclusive, &update_ctid, update_xmax); + } if (copyTuple == NULL) { break; } @@ -1712,7 +1832,11 @@ SelectForUpdateFusion::SelectForUpdateFusion( /* get limit num */ if (IsA(m_planstmt->planTree, Limit)) { Limit* limit = (Limit*)m_planstmt->planTree; - node = (IndexScan*)m_planstmt->planTree->lefttree->lefttree; + if (IsA(m_planstmt->planTree->lefttree->lefttree, PartIterator)) { + node = (IndexScan*)m_planstmt->planTree->lefttree->lefttree->lefttree; + } else { + node = (IndexScan*)m_planstmt->planTree->lefttree->lefttree; + } if (limit->limitOffset != NULL && IsA(limit->limitOffset, Const) && !((Const*)limit->limitOffset)->constisnull) { m_limitOffset = DatumGetInt64(((Const*)limit->limitOffset)->constvalue); @@ -1721,7 +1845,11 @@ SelectForUpdateFusion::SelectForUpdateFusion( m_limitCount = DatumGetInt64(((Const*)limit->limitCount)->constvalue); } } else { - node = (IndexScan*)m_planstmt->planTree->lefttree; + if (IsA(m_planstmt->planTree->lefttree, PartIterator)) { + node = (IndexScan*)m_planstmt->planTree->lefttree->lefttree; + } else { + node = (IndexScan*)m_planstmt->planTree->lefttree; + } } List* targetList = node->scan.plan.targetlist; @@ -1790,7 +1918,8 @@ bool SelectForUpdateFusion::execute(long max_rows, char* completionTag) m_scan->refreshParameter(m_outParams == NULL ? m_params : m_outParams); m_scan->Init(max_rows); } - Relation rel = m_scan->m_rel; + Relation rel = (m_scan->m_parentRel == NULL ? m_scan->m_rel : m_scan->m_parentRel); + Relation partRel = NULL; Relation bucket_rel = NULL; int2 bucketid = InvalidBktId; @@ -1803,6 +1932,10 @@ bool SelectForUpdateFusion::execute(long max_rows, char* completionTag) ExecOpenIndices(result_rel_info, false); } + if (RELATION_IS_PARTITIONED(rel)) { + partRel = m_scan->m_rel; + } + /************************************** * step 2: begin scan and update xmax * **************************************/ @@ -1857,15 +1990,25 @@ bool SelectForUpdateFusion::execute(long max_rows, char* completionTag) false); /* don't pfree this pointer */ slot_getsomeattrs(m_reslot, m_tupDesc->natts); - - result = heap_lock_tuple(bucket_rel == NULL ? rel : bucket_rel, - tuple, - &buffer, - &update_ctid, - &update_xmax, - m_estate->es_output_cid, - LockTupleExclusive, - false); + if (RELATION_IS_PARTITIONED(rel)) { + result = heap_lock_tuple(bucket_rel == NULL ? partRel : bucket_rel, + tuple, + &buffer, + &update_ctid, + &update_xmax, + m_estate->es_output_cid, + LockTupleExclusive, + false); + } else { + result = heap_lock_tuple(bucket_rel == NULL ? rel : bucket_rel, + tuple, + &buffer, + &update_ctid, + &update_xmax, + m_estate->es_output_cid, + LockTupleExclusive, + false); + } ReleaseBuffer(buffer); if (result == HeapTupleSelfUpdated) { @@ -1897,11 +2040,19 @@ bool SelectForUpdateFusion::execute(long max_rows, char* completionTag) bool* isnullfornew = NULL; Datum* valuesfornew = NULL; HeapTuple copyTuple; - copyTuple = EvalPlanQualFetch(m_estate, - bucket_rel == NULL ? rel : bucket_rel, - LockTupleExclusive, - &update_ctid, - update_xmax); + if (RELATION_IS_PARTITIONED(rel)) { + copyTuple = EvalPlanQualFetch(m_estate, + bucket_rel == NULL ? partRel : bucket_rel, + LockTupleExclusive, + &update_ctid, + update_xmax); + } else { + copyTuple = EvalPlanQualFetch(m_estate, + bucket_rel == NULL ? rel : bucket_rel, + LockTupleExclusive, + &update_ctid, + update_xmax); + } if (copyTuple == NULL) { break; } diff --git a/src/gausskernel/runtime/executor/opfusion_scan.cpp b/src/gausskernel/runtime/executor/opfusion_scan.cpp index 67c316608..d83244913 100644 --- a/src/gausskernel/runtime/executor/opfusion_scan.cpp +++ b/src/gausskernel/runtime/executor/opfusion_scan.cpp @@ -36,6 +36,7 @@ ScanFusion::ScanFusion(ParamListInfo params, PlannedStmt* planstmt) m_params = params; m_planstmt = planstmt; m_rel = NULL; + m_parentRel = NULL; m_tupDesc = NULL; m_reslot = NULL; m_direction = NULL; @@ -335,6 +336,8 @@ IndexScanFusion::IndexScanFusion(IndexScan* node, PlannedStmt* planstmt, ParamLi m_values = NULL; m_index = NULL; m_epq_indexqual = NULL; + m_parentRel = NULL; + m_partRel = NULL; m_node = node; m_keyInit = false; @@ -372,13 +375,30 @@ IndexScanFusion::IndexScanFusion(IndexScan* node, PlannedStmt* planstmt, ParamLi i++; } } + if (m_node->scan.isPartTbl) { + Oid parentRelOid = getrelid(m_node->scan.scanrelid, planstmt->rtable); + m_parentRel = heap_open(parentRelOid, AccessShareLock); + if (params != NULL) { + Param* paramArg = m_node->scan.pruningInfo->paramArg; + Assert(paramArg != NULL); + m_reloid= getPartitionOidByParam(m_parentRel, paramArg, + &(params->params[paramArg->paramid - 1])); + } else { + Assert(list_length(m_node->scan.pruningInfo->ls_rangeSelectedPartitions) != 0); + int partId = lfirst_int(list_head(m_node->scan.pruningInfo->ls_rangeSelectedPartitions)); + m_reloid = getPartitionOidFromSequence(m_parentRel, partId); + } - m_reloid = getrelid(m_node->scan.scanrelid, planstmt->rtable); + m_partRel = partitionOpen(m_parentRel, m_reloid, AccessShareLock); + m_rel = partitionGetRelation(m_parentRel, m_partRel); + } else { + m_reloid = getrelid(m_node->scan.scanrelid, planstmt->rtable); + m_rel = heap_open(m_reloid, AccessShareLock); + } m_targetList = m_node->scan.plan.targetlist; m_tupDesc = ExecCleanTypeFromTL(m_targetList, false); m_direction = (ScanDirection*)palloc0(sizeof(ScanDirection)); - m_rel = heap_open(m_reloid, AccessShareLock); Relation rel = m_rel; m_attrno = (int16*)palloc(m_tupDesc->natts * sizeof(int16)); m_values = (Datum*)palloc(RelationGetDescr(rel)->natts * sizeof(Datum)); @@ -386,12 +406,39 @@ IndexScanFusion::IndexScanFusion(IndexScan* node, PlannedStmt* planstmt, ParamLi m_isnull = (bool*)palloc(RelationGetDescr(rel)->natts * sizeof(bool)); m_tmpisnull = (bool*)palloc(m_tupDesc->natts * sizeof(bool)); setAttrNo(); - heap_close(m_rel, AccessShareLock); + + if (m_node->scan.isPartTbl) { + partitionClose(m_parentRel, m_partRel, AccessShareLock); + releaseDummyRelation(&m_rel); + heap_close(m_parentRel, AccessShareLock); + } else { + heap_close(m_rel, AccessShareLock); + } } void IndexScanFusion::Init(long max_rows) { - m_index = index_open(m_node->indexid, AccessShareLock); + if (m_node->scan.isPartTbl) { + Oid parent_relOid = getrelid(m_node->scan.scanrelid, m_planstmt->rtable); + m_parentRel = heap_open(parent_relOid, AccessShareLock); + + /* get partition relation */ + m_partRel = partitionOpen(m_parentRel, m_reloid, AccessShareLock); + PartitionGetPartIndexList(m_partRel); + m_rel = partitionGetRelation(m_parentRel, m_partRel); + + /* get partition index */ + Oid parentIndexOid = m_node->indexid; + Relation parentIndex = relation_open(parentIndexOid, AccessShareLock); + Oid partIndexOid = lfirst_int(list_head(m_rel->rd_indexlist)); + Partition partIndex = partitionOpen(parentIndex, partIndexOid, AccessShareLock); + m_index = partitionGetRelation(parentIndex, partIndex); + partitionClose(parentIndex, partIndex, AccessShareLock); + relation_close(parentIndex, AccessShareLock); + } else { + m_rel = heap_open(m_reloid, AccessShareLock); + m_index = index_open(m_node->indexid, AccessShareLock); + } if (unlikely(!m_keyInit)) { IndexFusion::IndexBuildScanKey(m_node->indexqual); @@ -411,7 +458,6 @@ void IndexScanFusion::Init(long max_rows) *m_direction = NoMovementScanDirection; } - m_rel = heap_open(m_reloid, AccessShareLock); ScanState* scanstate = makeNode(ScanState); // need release scanstate->ps.plan = (Plan *)m_node; m_scandesc = (AbsIdxScanDesc)abs_idx_beginscan(m_rel, m_index, GetActiveSnapshot(), m_keyNum, 0, scanstate); // add scanstate pointer ? @@ -474,10 +520,20 @@ void IndexScanFusion::End(bool isCompleted) abs_idx_endscan(m_scandesc); } if (m_index != NULL) { - index_close(m_index, NoLock); + if (m_node->scan.isPartTbl) { + releaseDummyRelation(&m_index); + } else { + index_close(m_index, NoLock); + } } if (m_rel != NULL) { - heap_close(m_rel, NoLock); + if (m_node->scan.isPartTbl) { + partitionClose(m_parentRel, m_partRel, AccessShareLock); + releaseDummyRelation(&m_rel); + heap_close(m_parentRel, AccessShareLock); + } else { + heap_close(m_rel, NoLock); + } } } diff --git a/src/gausskernel/runtime/executor/opfusion_util.cpp b/src/gausskernel/runtime/executor/opfusion_util.cpp index bd192e3e9..507670e37 100755 --- a/src/gausskernel/runtime/executor/opfusion_util.cpp +++ b/src/gausskernel/runtime/executor/opfusion_util.cpp @@ -215,6 +215,36 @@ const char *getBypassReason(FusionType result) break; } + case NOBYPASS_ZERO_PARTITION: { + return "Bypass not support query in zero partition"; + break; + } + + case NOBYPASS_MULTI_PARTITION: { + return "Bypass not support query in multiple partitions"; + break; + } + + case NOBYPASS_EXP_NOT_SUPPORT_IN_PARTITION: { + return "Bypass not executed because the expression of query is not support in partition table"; + break; + } + + case NO_BYPASS_PARTITIONKEY_IS_NULL: { + return "Bypass not executed because the partition key is null"; + break; + } + + case NOBYPASS_NO_UPDATE_PARTITIONKEY: { + return "Bypass not support update the partition key"; + break; + } + + case NOBYPASS_NO_INCLUDING_PARTITIONKEY: { + return "Bypass not executed because the partition key is not in the parameters"; + break; + } + default: { Assert(0); ereport(ERROR, @@ -617,6 +647,26 @@ FusionType checkFusionNestLoop(NestLoop *node, ParamListInfo params) return NESTLOOP_INDEX_FUSION; } +template FusionType checkFusionPartitionNumber(Node *node, FusionType ftype) +{ + if (isonlyindex) { + IndexOnlyScan* scan = (IndexOnlyScan *)node; + if (scan->scan.itrs == 0) { + return NOBYPASS_ZERO_PARTITION; + } else if (scan->scan.itrs > 1) { + return NOBYPASS_MULTI_PARTITION; + } + } else { + IndexScan* scan = (IndexScan *)node; + if (scan->scan.itrs == 0) { + return NOBYPASS_ZERO_PARTITION; + } else if (scan->scan.itrs > 1) { + return NOBYPASS_MULTI_PARTITION; + } + } + return ftype; +} + FusionType getSelectFusionType(List *stmt_list, ParamListInfo params) { FusionType ftype = SELECT_FUSION; @@ -702,7 +752,13 @@ FusionType getSelectFusionType(List *stmt_list, ParamListInfo params) top_plan = top_plan->lefttree; } #endif - + if (IsA(top_plan, PartIterator)) { + if (u_sess->attr.attr_sql.enable_partition_opfusion) { + top_plan = top_plan->lefttree; + } else { + return NONE_FUSION; + } + } /* check for indexscan or indexonlyscan */ if ((IsA(top_plan, IndexScan) || IsA(top_plan, IndexOnlyScan)) && top_plan->lefttree == NULL) { FusionType ttype; @@ -718,6 +774,18 @@ FusionType getSelectFusionType(List *stmt_list, ParamListInfo params) } else { return NOBYPASS_NO_INDEXSCAN; } + /* check for the number of partitions */ + if (IsA(top_plan, IndexScan)) { + IndexScan* scan = (IndexScan *)top_plan; + if (scan->scan.isPartTbl == true) { + ftype = checkFusionPartitionNumber((Node *)scan, ftype); + } + } else { + IndexOnlyScan* scan = (IndexOnlyScan *)top_plan; + if (scan->scan.isPartTbl == true) { + ftype = checkFusionPartitionNumber((Node *)scan, ftype); + } + } return ftype; } @@ -741,8 +809,8 @@ FusionType checkTargetlist(List *targetList, FusionType ftype) bool checkDMLRelation(Relation rel, PlannedStmt *plannedstmt) { if (rel->rd_rel->relkind != RELKIND_RELATION || rel->rd_rel->relhasrules || rel->rd_rel->relhastriggers || - rel->rd_rel->relhasoids || rel->rd_rel->relhassubclass || RelationIsPartitioned(rel) || - RelationIsColStore(rel) || RelationInRedistribute(rel) || plannedstmt->hasReturning) { + rel->rd_rel->relhasoids || rel->rd_rel->relhassubclass || RelationIsColStore(rel) || + RelationInRedistribute(rel) || plannedstmt->hasReturning) { return true; } return false; @@ -782,6 +850,10 @@ FusionType getInsertFusionType(List *stmt_list, ParamListInfo params) Index res_rel_idx = linitial_int(plannedstmt->resultRelations); Oid relid = getrelid(res_rel_idx, plannedstmt->rtable); Relation rel = heap_open(relid, AccessShareLock); + if (RelationIsPartitioned(rel) && !u_sess->attr.attr_sql.enable_partition_opfusion) { + heap_close(rel, AccessShareLock); + return NONE_FUSION; + } for (int i = 0; i < rel->rd_att->natts; i++) { if (rel->rd_att->attrs[i]->attisdropped) { @@ -835,13 +907,21 @@ FusionType getUpdateFusionType(List *stmt_list, ParamListInfo params) if (list_length(node->plans) != 1) { return NOBYPASS_NO_SIMPLE_PLAN; } + Plan *updatePlan = (Plan *)linitial(node->plans); + if (IsA(updatePlan, PartIterator)) { + if (u_sess->attr.attr_sql.enable_partition_opfusion) { + updatePlan = updatePlan->lefttree; + } else { + return NONE_FUSION; + } + } - if (!IsA(linitial(node->plans), IndexScan)) { + if (!IsA(updatePlan, IndexScan)) { return NOBYPASS_NO_INDEXSCAN; } /* check index scan */ - FusionType ttype = checkFusionIndexScan((Node *)linitial(node->plans), params); + FusionType ttype = checkFusionIndexScan((Node *)updatePlan, params); /* check failed */ if (ttype > BYPASS_OK) { return ttype; @@ -855,12 +935,25 @@ FusionType getUpdateFusionType(List *stmt_list, ParamListInfo params) heap_close(rel, AccessShareLock); return NOBYPASS_DML_RELATION_NOT_SUPPORT; } + IndexScan *indexscan = (IndexScan *)updatePlan; + if (RELATION_IS_PARTITIONED(rel) && !(indexscan->scan.isPartTbl)) { + heap_close(rel, AccessShareLock); + return NOBYPASS_DML_RELATION_NOT_SUPPORT; + } heap_close(rel, AccessShareLock); /* check target list */ - IndexScan *indexscan = (IndexScan *)linitial(node->plans); + if (node->partKeyUpdated == true) { + return NOBYPASS_NO_UPDATE_PARTITIONKEY; + } List *targetlist = indexscan->scan.plan.targetlist; - return checkTargetlist(targetlist, ftype); + ftype = checkTargetlist(targetlist, ftype); + + /* check the number of partitions */ + if (indexscan->scan.isPartTbl) { + ftype = checkFusionPartitionNumber((Node *)indexscan, ftype); + } + return ftype; } FusionType getDeleteFusionType(List *stmt_list, ParamListInfo params) @@ -883,18 +976,26 @@ FusionType getDeleteFusionType(List *stmt_list, ParamListInfo params) if (list_length(node->plans) != 1) { return NOBYPASS_NO_SIMPLE_PLAN; } - - if (!IsA(linitial(node->plans), IndexScan)) { + Plan *deletePlan = (Plan *)linitial(node->plans); + if (IsA(deletePlan, PartIterator)) { + if (u_sess->attr.attr_sql.enable_partition_opfusion) { + deletePlan = deletePlan->lefttree; + } else { + return NONE_FUSION; + } + } + if (!IsA(deletePlan, IndexScan)) { return NOBYPASS_NO_INDEXSCAN; } /* check index scan */ - FusionType ttype = checkFusionIndexScan((Node *)linitial(node->plans), params); + FusionType ttype = checkFusionIndexScan((Node *)deletePlan, params); /* check failed */ if (ttype > BYPASS_OK) { return ttype; } /* check relation */ + IndexScan* indexscan = (IndexScan *)deletePlan; Index res_rel_idx = linitial_int(plannedstmt->resultRelations); Oid relid = getrelid(res_rel_idx, plannedstmt->rtable); Relation rel = heap_open(relid, AccessShareLock); @@ -904,6 +1005,11 @@ FusionType getDeleteFusionType(List *stmt_list, ParamListInfo params) } heap_close(rel, AccessShareLock); + /* check the number of partitions */ + if (indexscan->scan.isPartTbl) { + ftype = checkFusionPartitionNumber((Node *)indexscan, ftype); + } + return ftype; } diff --git a/src/gausskernel/storage/access/heap/heapam.cpp b/src/gausskernel/storage/access/heap/heapam.cpp index a81e5c44e..8e7e48ac7 100644 --- a/src/gausskernel/storage/access/heap/heapam.cpp +++ b/src/gausskernel/storage/access/heap/heapam.cpp @@ -1759,20 +1759,6 @@ void heap_rescan(HeapScanDesc scan, ScanKey key) * reinitialize scan descriptor */ initscan(scan, key, true); - - /* - * reset parallel scan, if present - */ - if (scan->rs_parallel != NULL) { - ParallelHeapScanDesc parallel_scan; - - /* - * Caller is responsible for making sure that all workers have - * finished the scan before calling this. - */ - parallel_scan = scan->rs_parallel; - pg_atomic_write_u64(¶llel_scan->phs_nallocated, 0); - } } /* ---------------- @@ -1913,6 +1899,18 @@ void heap_parallelscan_initialize(ParallelHeapScanDesc target, Size pscan_len, R } } +/* ---------------- + * heap_parallelscan_reinitialize - reset a parallel scan + * + * Call this in the leader process. Caller is responsible for + * making sure that all workers have finished the scan beforehand. + * ---------------- + */ +void heap_parallelscan_reinitialize(ParallelHeapScanDesc parallel_scan) +{ + pg_atomic_write_u64(¶llel_scan->phs_nallocated, 0); +} + /* ---------------- * heap_beginscan_parallel - join a parallel scan * diff --git a/src/gausskernel/storage/access/nbtree/nbtsearch.cpp b/src/gausskernel/storage/access/nbtree/nbtsearch.cpp index 603ca0e73..7be072900 100755 --- a/src/gausskernel/storage/access/nbtree/nbtsearch.cpp +++ b/src/gausskernel/storage/access/nbtree/nbtsearch.cpp @@ -747,7 +747,7 @@ bool _bt_first(IndexScanDesc scan, ScanDirection dir) break; /* wrong direction, can't use it */ if (subkey->sk_flags & SK_ISNULL) break; /* can't use null keys */ - Assert(keysCount < INDEX_MAX_KEYS); + Assert(keysCount < INDEX_MAX_KEYS); scankeys[keysCount] = *subkey; keysCount++; if (subkey->sk_flags & SK_ROW_END) { diff --git a/src/gausskernel/storage/file/buffile.cpp b/src/gausskernel/storage/file/buffile.cpp index 2fe1a5701..2d71d2d4c 100644 --- a/src/gausskernel/storage/file/buffile.cpp +++ b/src/gausskernel/storage/file/buffile.cpp @@ -462,6 +462,7 @@ static void BufFileLoadBuffer(BufFile* file) if (file->nbytes < 0) { file->nbytes = 0; + ereport(ERROR, (errcode_for_file_access(), errmsg("could not read file \"%s\": %m", FilePathName(thisfile)))); } file->offsets[file->curFile] += file->nbytes; diff --git a/src/gausskernel/storage/ipc/Makefile b/src/gausskernel/storage/ipc/Makefile index 8bdc7c560..25b57dbd4 100644 --- a/src/gausskernel/storage/ipc/Makefile +++ b/src/gausskernel/storage/ipc/Makefile @@ -16,7 +16,7 @@ ifneq "$(MAKECMDGOALS)" "clean" endif endif endif -OBJS = ipc.o ipci.o pmsignal.o procarray.o procsignal.o shmem.o shmqueue.o \ +OBJS = ipc.o ipci.o pmsignal.o pg_barrier.o procarray.o procsignal.o shmem.o shmqueue.o \ sinval.o sinvaladt.o standby.o shm_mq.o shm_toc.o dsm.o include $(top_srcdir)/src/gausskernel/common.mk \ No newline at end of file diff --git a/src/gausskernel/storage/ipc/pg_barrier.cpp b/src/gausskernel/storage/ipc/pg_barrier.cpp new file mode 100644 index 000000000..23faf501e --- /dev/null +++ b/src/gausskernel/storage/ipc/pg_barrier.cpp @@ -0,0 +1,316 @@ +/* ------------------------------------------------------------------------- + * + * pg_barrier.c + * Barriers for synchronizing cooperating processes. + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * From Wikipedia[1]: "In parallel computing, a barrier is a type of + * synchronization method. A barrier for a group of threads or processes in + * the source code means any thread/process must stop at this point and cannot + * proceed until all other threads/processes reach this barrier." + * + * This implementation of barriers allows for static sets of participants + * known up front, or dynamic sets of participants which processes can join or + * leave at any time. In the dynamic case, a phase number can be used to + * track progress through a parallel algorithm, and may be necessary to + * synchronize with the current phase of a multi-phase algorithm when a new + * participant joins. In the static case, the phase number is used + * internally, but it isn't strictly necessary for client code to access it + * because the phase can only advance when the declared number of participants + * reaches the barrier, so client code should be in no doubt about the current + * phase of computation at all times. + * + * Consider a parallel algorithm that involves separate phases of computation + * A, B and C where the output of each phase is needed before the next phase + * can begin. + * + * In the case of a static barrier initialized with 4 participants, each + * participant works on phase A, then calls BarrierArriveAndWait to wait until + * all 4 participants have reached that point. When BarrierArriveAndWait + * returns control, each participant can work on B, and so on. Because the + * barrier knows how many participants to expect, the phases of computation + * don't need labels or numbers, since each process's program counter implies + * the current phase. Even if some of the processes are slow to start up and + * begin running phase A, the other participants are expecting them and will + * patiently wait at the barrier. The code could be written as follows: + * + * perform_a(); + * BarrierArriveAndWait(&barrier, ...); + * perform_b(); + * BarrierArriveAndWait(&barrier, ...); + * perform_c(); + * BarrierArriveAndWait(&barrier, ...); + * + * If the number of participants is not known up front, then a dynamic barrier + * is needed and the number should be set to zero at initialization. New + * complications arise because the number necessarily changes over time as + * participants attach and detach, and therefore phases B, C or even the end + * of processing may be reached before any given participant has started + * running and attached. Therefore the client code must perform an initial + * test of the phase number after attaching, because it needs to find out + * which phase of the algorithm has been reached by any participants that are + * already attached in order to synchronize with that work. Once the program + * counter or some other representation of current progress is synchronized + * with the barrier's phase, normal control flow can be used just as in the + * static case. Our example could be written using a switch statement with + * cases that fall-through, as follows: + * + * phase = BarrierAttach(&barrier); + * switch (phase) + * { + * case PHASE_A: + * perform_a(); + * BarrierArriveAndWait(&barrier, ...); + * case PHASE_B: + * perform_b(); + * BarrierArriveAndWait(&barrier, ...); + * case PHASE_C: + * perform_c(); + * BarrierArriveAndWait(&barrier, ...); + * } + * BarrierDetach(&barrier); + * + * Static barriers behave similarly to POSIX's pthread_barrier_t. Dynamic + * barriers behave similarly to Java's java.util.concurrent.Phaser. + * + * [1] https://en.wikipedia.org/wiki/Barrier_(computer_science) + * + * IDENTIFICATION + * src/backend/storage/ipc/barrier.c + * + * ------------------------------------------------------------------------- + */ + +#include "postgres.h" +#include "storage/pg_barrier.h" + +static inline bool BarrierDetachImpl(Barrier* barrier, bool arrive); + +/* + * Initialize this barrier. To use a static party size, provide the number of + * participants to wait for at each phase indicating that that number of + * backends is implicitly attached. To use a dynamic party size, specify zero + * here and then use BarrierAttach() and + * BarrierDetach()/BarrierArriveAndDetach() to register and deregister + * participants explicitly. + */ +void BarrierInit(Barrier* barrier, int participants) +{ + SpinLockInit(&barrier->mutex); + barrier->participants = participants; + barrier->arrived = 0; + barrier->phase = 0; + barrier->elected = 0; + barrier->static_party = participants > 0; + ConditionVariableInit(&barrier->condition_variable); +} + +/* + * Arrive at this barrier, wait for all other attached participants to arrive + * too and then return. Increments the current phase. The caller must be + * attached. + * + * While waiting, pg_stat_activity shows a wait_event_class and wait_event + * controlled by the wait_event_info passed in, which should be a value from + * from one of the WaitEventXXX enums defined in pgstat.h. + * + * Return true in one arbitrarily chosen participant. Return false in all + * others. The return code can be used to elect one participant to execute a + * phase of work that must be done serially while other participants wait. + */ +bool BarrierArriveAndWaitStatus(Barrier* barrier) +{ + bool release = false; + bool elected; + int startPhase; + int nextPhase; + + SpinLockAcquire(&barrier->mutex); + startPhase = barrier->phase; + nextPhase = startPhase + 1; + ++barrier->arrived; + if (barrier->arrived == barrier->participants) { + release = true; + barrier->arrived = 0; + barrier->phase = nextPhase; + barrier->elected = nextPhase; + } + SpinLockRelease(&barrier->mutex); + + /* + * If we were the last expected participant to arrive, we can release our + * peers and return true to indicate that this backend has been elected to + * perform any serial work. + */ + if (release) { + ConditionVariableBroadcast(&barrier->condition_variable); + return true; + } + + /* + * Otherwise we have to wait for the last participant to arrive and + * advance the phase. + */ + elected = false; + (void)pthread_mutex_lock(&(barrier->condition_variable.mutex)); + for (;;) { + CHECK_FOR_INTERRUPTS(); + /* + * We know that phase must either be startPhase, indicating that we + * need to keep waiting, or nextPhase, indicating that the last + * participant that we were waiting for has either arrived or detached + * so that the next phase has begun. The phase cannot advance any + * further than that without this backend's participation, because + * this backend is attached. + */ + SpinLockAcquire(&barrier->mutex); + Assert(barrier->phase == startPhase || barrier->phase == nextPhase); + release = barrier->phase == nextPhase; + if (release && barrier->elected != nextPhase) { + /* + * Usually the backend that arrives last and releases the other + * backends is elected to return true (see above), so that it can + * begin processing serial work while it has a CPU timeslice. + * However, if the barrier advanced because someone detached, then + * one of the backends that is awoken will need to be elected. + */ + barrier->elected = barrier->phase; + elected = true; + } + SpinLockRelease(&barrier->mutex); + if (release) { + break; + } + struct timespec ts; + (void)clock_gettime(CLOCK_REALTIME, &ts); + ts.tv_sec += 1; + (void)pthread_cond_timedwait(&(barrier->condition_variable.cond), &(barrier->condition_variable.mutex), &ts); + } + (void)pthread_mutex_unlock(&(barrier->condition_variable.mutex)); + return elected; +} + +/* + * Arrive at this barrier, but detach rather than waiting. Returns true if + * the caller was the last to detach. + */ +bool BarrierArriveAndDetach(Barrier* barrier) +{ + return BarrierDetachImpl(barrier, true); +} + +/* + * Attach to a barrier. All waiting participants will now wait for this + * participant to call BarrierArriveAndWait(), BarrierDetach() or + * BarrierArriveAndDetach(). Return the current phase. + */ +int BarrierAttach(Barrier* barrier) +{ + int phase; + + Assert(!barrier->static_party); + + SpinLockAcquire(&barrier->mutex); + ++barrier->participants; + phase = barrier->phase; + SpinLockRelease(&barrier->mutex); + + return phase; +} + +/* + * Detach from a barrier. This may release other waiters from BarrierWait and + * advance the phase if they were only waiting for this backend. Return true + * if this participant was the last to detach. + */ +bool BarrierDetach(Barrier* barrier) +{ + return BarrierDetachImpl(barrier, false); +} + +/* + * Return the current phase of a barrier. The caller must be attached. + */ +int BarrierPhase(Barrier* barrier) +{ + /* + * It is OK to read barrier->phase without locking, because it can't + * change without us (we are attached to it), and we executed a memory + * barrier when we either attached or participated in changing it last + * time. + */ + return barrier->phase; +} + +/* + * Return an instantaneous snapshot of the number of participants currently + * attached to this barrier. For debugging purposes only. + */ +int BarrierParticipants(Barrier* barrier) +{ + int participants; + + SpinLockAcquire(&barrier->mutex); + participants = barrier->participants; + SpinLockRelease(&barrier->mutex); + return participants; +} + +/* + * Detach from a barrier. If 'arrive' is true then also increment the phase + * if there are no other participants. If there are other participants + * waiting, then the phase will be advanced and they'll be released if they + * were only waiting for the caller. Return true if this participant was the + * last to detach. + */ +static inline bool BarrierDetachImpl(Barrier* barrier, bool arrive) +{ + bool release; + bool last; + + Assert(!barrier->static_party); + + SpinLockAcquire(&barrier->mutex); + Assert(barrier->participants > 0); + --barrier->participants; + + /* + * If any other participants are waiting and we were the last participant + * waited for, release them. If no other participants are waiting, but + * this is a BarrierArriveAndDetach() call, then advance the phase too. + */ + if ((arrive || barrier->participants > 0) && barrier->arrived == barrier->participants) { + release = true; + barrier->arrived = 0; + ++barrier->phase; + } else { + release = false; + } + last = barrier->participants == 0; + SpinLockRelease(&barrier->mutex); + if (release) { + ConditionVariableBroadcast(&barrier->condition_variable); + } + + return last; +} + +void ConditionVariableInit(ConditionVariable* cv) +{ + (void)pthread_mutex_init(&(cv->mutex), NULL); + (void)pthread_cond_init(&(cv->cond), NULL); +} + +/* + * Wake up all sleeping processes. + * + * The return value indicates the number of processes we woke. + */ +void ConditionVariableBroadcast(ConditionVariable* cv) +{ + (void)pthread_mutex_lock(&(cv->mutex)); + (void)pthread_cond_broadcast(&(cv->cond)); + (void)pthread_mutex_unlock(&(cv->mutex)); +} diff --git a/src/gausskernel/storage/lmgr/lwlock.cpp b/src/gausskernel/storage/lmgr/lwlock.cpp index 61305e244..be12c0d82 100755 --- a/src/gausskernel/storage/lmgr/lwlock.cpp +++ b/src/gausskernel/storage/lmgr/lwlock.cpp @@ -157,7 +157,8 @@ static const char *BuiltinTrancheNames[] = { "GlobalTempTableControl", "PLdebugger", "SharedTupleStore", - "parallel_append" + "parallel_append", + "ParallelHashJoinLock" }; static void RegisterLWLockTranches(void); diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h index aac08d89b..413d2ad12 100644 --- a/src/include/access/heapam.h +++ b/src/include/access/heapam.h @@ -105,6 +105,7 @@ extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction); extern Size heap_parallelscan_estimate(Snapshot snapshot); extern void heap_parallelscan_initialize(ParallelHeapScanDesc target, Size pscan_len, Relation relation, Snapshot snapshot); +extern void heap_parallelscan_reinitialize(ParallelHeapScanDesc parallel_scan); extern HeapScanDesc heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan); extern void heap_init_parallel_seqscan(HeapScanDesc scan, int32 dop, ScanDirection dir); diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h index 13725311b..dc21d39bf 100644 --- a/src/include/executor/execParallel.h +++ b/src/include/executor/execParallel.h @@ -36,7 +36,7 @@ extern ParallelExecutorInfo *ExecInitParallelPlan(PlanState *planstate, EState * extern void ExecParallelCreateReaders(ParallelExecutorInfo *pei, TupleDesc tupDesc); extern void ExecParallelFinish(ParallelExecutorInfo *pei); extern void ExecParallelCleanup(ParallelExecutorInfo *pei); -extern void ExecParallelReinitialize(ParallelExecutorInfo *pei); +extern void ExecParallelReinitialize(PlanState *planstate, ParallelExecutorInfo *pei); extern void ParallelQueryMain(void *seg); #endif /* EXECPARALLEL_H */ diff --git a/src/include/executor/hashjoin.h b/src/include/executor/hashjoin.h index a3d1ae654..eb5cb83e7 100755 --- a/src/include/executor/hashjoin.h +++ b/src/include/executor/hashjoin.h @@ -15,7 +15,12 @@ #define HASHJOIN_H #include "nodes/execnodes.h" +#include "utils/atomic.h" +#include "utils/dsa.h" #include "storage/buffile.h" +#include "storage/lwlock.h" +#include "storage/pg_barrier.h" + /* ---------------------------------------------------------------- * hash-join hash table structures @@ -62,9 +67,13 @@ /* typedef struct HashJoinTableData *HashJoinTable; */ typedef struct HashJoinTupleData { - struct HashJoinTupleData* next; /* link to next tuple in same bucket */ - uint32 hashvalue; /* tuple's hash code */ - /* Tuple data, in MinimalTuple format, follows on a MAXALIGN boundary */ + /* link to next tuple in same bucket */ + union { + HashJoinTuple unshared; + HashJoinTuple shared; + } next; + uint32 hashvalue; /* tuple's hash code */ + /* Tuple data, in MinimalTuple format, follows on a MAXALIGN boundary */ } HashJoinTupleData; #define HJTUPLE_OVERHEAD MAXALIGN(sizeof(HashJoinTupleData)) @@ -108,23 +117,161 @@ typedef struct HashMemoryChunkData { size_t maxlen; /* size of the buffer holding the tuples */ size_t used; /* number of buffer bytes already used */ - struct HashMemoryChunkData* next; /* pointer to the next chunk (linked list) */ - + /* pointer to the next chunk (linked list) */ + union { + HashMemoryChunkData* unshared; + HashMemoryChunkData* shared; + } next; char data[FLEXIBLE_ARRAY_MEMBER]; /* buffer allocated at the end */ } HashMemoryChunkData; typedef struct HashMemoryChunkData* HashMemoryChunk; #define HASH_CHUNK_SIZE (32 * 1024L) +#define HASH_CHUNK_HEADER_SIZE (offsetof(HashMemoryChunkData, data)) #define HASH_CHUNK_THRESHOLD (HASH_CHUNK_SIZE / 4) +/* + * For each batch of a Parallel Hash Join, we have a ParallelHashJoinBatch + * object in shared memory to coordinate access to it. Since they are + * followed by variable-sized objects, they are arranged in contiguous memory + * but not accessed directly as an array. + */ +typedef struct ParallelHashJoinBatch { + void* buckets; /* array of hash table buckets */ + Barrier batch_barrier; /* synchronization for joining this batch */ + + HashMemoryChunk chunks; /* chunks of tuples loaded */ + size_t size; /* size of buckets + chunks in memory */ + size_t estimated_size; /* size of buckets + chunks while writing */ + size_t ntuples; /* number of tuples loaded */ + size_t old_ntuples; /* number of tuples before repartitioning */ + bool space_exhausted; + + /* + * Variable-sized SharedTuplestore objects follow this struct in memory. + * See the accessor macros below. + */ +} ParallelHashJoinBatch; + +/* Accessor for inner batch tuplestore following a ParallelHashJoinBatch. */ +#define ParallelHashJoinBatchInner(batch) \ + ((SharedTuplestore*)((char*)(batch) + MAXALIGN(sizeof(ParallelHashJoinBatch)))) + +/* Accessor for outer batch tuplestore following a ParallelHashJoinBatch. */ +#define ParallelHashJoinBatchOuter(batch, nparticipants) \ + ((SharedTuplestore*)((char*)ParallelHashJoinBatchInner(batch) + MAXALIGN(sts_estimate(nparticipants)))) + +/* Total size of a ParallelHashJoinBatch and tuplestores. */ +#define EstimateParallelHashJoinBatch(hashtable) \ + (MAXALIGN(sizeof(ParallelHashJoinBatch)) + MAXALIGN(sts_estimate((hashtable)->parallel_state->nparticipants)) * 2) + +/* Accessor for the nth ParallelHashJoinBatch given the base. */ +#define NthParallelHashJoinBatch(base, n) \ + ((ParallelHashJoinBatch*)((char*)(base) + EstimateParallelHashJoinBatch(hashtable) * (n))) + +/* + * Each backend requires a small amount of per-batch state to interact with + * each ParalellHashJoinBatch. + */ +typedef struct ParallelHashJoinBatchAccessor { + ParallelHashJoinBatch* shared; /* pointer to shared state */ + + /* Per-backend partial counters to reduce contention. */ + size_t preallocated; /* pre-allocated space for this backend */ + size_t ntuples; /* number of tuples */ + size_t size; /* size of partition in memory */ + size_t estimated_size; /* size of partition on disk */ + size_t old_ntuples; /* how many tuples before repartioning? */ + bool at_least_one_chunk; /* has this backend allocated a chunk? */ + + bool done; /* flag to remember that a batch is done */ + SharedTuplestoreAccessor* inner_tuples; + SharedTuplestoreAccessor* outer_tuples; +} ParallelHashJoinBatchAccessor; + +/* + * While hashing the inner relation, any participant might determine that it's + * time to increase the number of buckets to reduce the load factor or batches + * to reduce the memory size. This is indicated by setting the growth flag to + * these values. + */ +typedef enum ParallelHashGrowth { + /* The current dimensions are sufficient. */ + PHJ_GROWTH_OK, + /* The load factor is too high, so we need to add buckets. */ + PHJ_GROWTH_NEED_MORE_BUCKETS, + /* The memory budget would be exhausted, so we need to repartition. */ + PHJ_GROWTH_NEED_MORE_BATCHES, + /* Repartitioning didn't help last time, so don't try to do that again. */ + PHJ_GROWTH_DISABLED +} ParallelHashGrowth; + +/* + * The shared state used to coordinate a Parallel Hash Join. This is stored + * in the DSM segment. + */ +typedef struct ParallelHashJoinState { + int plan_node_id; /* used to identify speicific plan */ + ParallelHashJoinBatch* batches; /* array of ParallelHashJoinBatch */ + ParallelHashJoinBatch* old_batches; /* previous generation during repartition */ + int nbatch; /* number of batches now */ + int old_nbatch; /* previous number of batches */ + int nbuckets; /* number of buckets */ + ParallelHashGrowth growth; /* control batch/bucket growth */ + HashMemoryChunk chunk_work_queue; /* chunk work queue */ + int nparticipants; + size_t space_allowed; + size_t total_tuples; /* total number of inner tuples */ + LWLock lock; /* lock protecting the above */ + + Barrier build_barrier; /* synchronization for the build phases */ + Barrier grow_batches_barrier; + Barrier grow_buckets_barrier; + pg_atomic_uint32 distributor; /* counter for load balancing */ + + SharedFileSet fileset; /* space for shared temporary files */ +} ParallelHashJoinState; + +/* The phases for building batches, used by build_barrier. */ +#define PHJ_BUILD_ELECTING 0 +#define PHJ_BUILD_ALLOCATING 1 +#define PHJ_BUILD_HASHING_INNER 2 +#define PHJ_BUILD_HASHING_OUTER 3 +#define PHJ_BUILD_DONE 4 + +/* The phases for probing each batch, used by for batch_barrier. */ +#define PHJ_BATCH_ELECTING 0 +#define PHJ_BATCH_ALLOCATING 1 +#define PHJ_BATCH_LOADING 2 +#define PHJ_BATCH_PROBING 3 +#define PHJ_BATCH_DONE 4 + +/* The phases of batch growth while hashing, for grow_batches_barrier. */ +#define PHJ_GROW_BATCHES_ELECTING 0 +#define PHJ_GROW_BATCHES_ALLOCATING 1 +#define PHJ_GROW_BATCHES_REPARTITIONING 2 +#define PHJ_GROW_BATCHES_DECIDING 3 +#define PHJ_GROW_BATCHES_FINISHING 4 +#define PHJ_GROW_BATCHES_PHASE(n) ((n) % 5) /* circular phases */ + +/* The phases of bucket growth while hashing, for grow_buckets_barrier. */ +#define PHJ_GROW_BUCKETS_ELECTING 0 +#define PHJ_GROW_BUCKETS_ALLOCATING 1 +#define PHJ_GROW_BUCKETS_REINSERTING 2 +#define PHJ_GROW_BUCKETS_PHASE(n) ((n) % 3) /* circular phases */ + typedef struct HashJoinTableData { int nbuckets; /* # buckets in the in-memory hash table */ int log2_nbuckets; /* its log2 (nbuckets must be a power of 2) */ - + /* buckets[i] is head of list of tuples in i'th in-memory bucket */ - struct HashJoinTupleData** buckets; - /* buckets array is per-batch storage, as are all the tuples */ + union { + /* unshared array is per-batch storage, as are all the tuples */ + HashJoinTuple* unshared; + /* shared array is per-query DSA area, as are all the tuples */ + HashJoinTuple* shared; + } buckets; /* buckets array is per-batch storage, as are all the tuples */ bool keepNulls; /* true to store unmatchable NULL tuples */ @@ -143,7 +290,7 @@ typedef struct HashJoinTableData { bool growEnabled; /* flag to shut off nbatch increases */ double totalTuples; /* # tuples obtained from inner plan */ - + double partialTuples; /* # tuples obtained from inner plan by me */ /* * These arrays are allocated for the life of the hash join, but only if * nbatch > 1. A file is opened only when we first write a tuple into it @@ -174,6 +321,13 @@ typedef struct HashJoinTableData { /* used for dense allocation of tuples (into linked chunks) */ HashMemoryChunk chunks; /* one list for the whole batch */ + + /* Shared and private state for Parallel Hash. */ + HashMemoryChunk current_chunk; /* this backend's current chunk */ + MemoryContext area; /* DSA area to allocate memory from */ + ParallelHashJoinState* parallel_state; + ParallelHashJoinBatchAccessor* batches; + void* current_chunk_shared; int64 width[2]; /* first records tuple count, second records total width */ bool causedBySysRes; /* the batch increase caused by system resources limit? */ int64 maxMem; /* batch auto spread mem */ diff --git a/src/include/executor/nodeHash.h b/src/include/executor/nodeHash.h index 734bf1910..8b874965e 100755 --- a/src/include/executor/nodeHash.h +++ b/src/include/executor/nodeHash.h @@ -14,9 +14,12 @@ #ifndef NODEHASH_H #define NODEHASH_H +#include "access/parallel.h" #include "nodes/execnodes.h" #include "nodes/relation.h" +struct SharedHashJoinBatch; + #define MIN_HASH_BUCKET_SIZE 32768 /* min bucketsize for hash join */ #define BUCKET_OVERHEAD 8 @@ -26,20 +29,30 @@ extern Node* MultiExecHash(HashState* node); extern void ExecEndHash(HashState* node); extern void ExecReScanHash(HashState* node); -extern HashJoinTable ExecHashTableCreate(Hash* node, List* hashOperators, bool keepNulls); +extern HashJoinTable ExecHashTableCreate(HashState* state, List* hashOperators, bool keepNulls); +extern void ExecParallelHashTableAlloc(HashJoinTable hashtable, int batchno); extern void ExecHashTableDestroy(HashJoinTable hashtable); +extern void ExecHashTableDetach(HashJoinTable hashtable); +extern void ExecHashTableDetachBatch(HashJoinTable hashtable); +extern void ExecParallelHashTableSetCurrentBatch(HashJoinTable hashtable, int batchno); +void ExecParallelHashUpdateSpacePeak(HashJoinTable hashtable, int batchno); + extern void ExecHashTableInsert(HashJoinTable hashtable, TupleTableSlot* slot, uint32 hashvalue, int planid, int dop, Instrumentation* instrument = NULL); +extern void ExecParallelHashTableInsert(HashJoinTable hashtable, TupleTableSlot* slot, uint32 hashvalue); +extern void ExecParallelHashTableInsertCurrentBatch(HashJoinTable hashtable, TupleTableSlot* slot, uint32 hashvalue); extern bool ExecHashGetHashValue(HashJoinTable hashtable, ExprContext* econtext, List* hashkeys, bool outer_tuple, bool keep_nulls, uint32* hashvalue); extern void ExecHashGetBucketAndBatch(HashJoinTable hashtable, uint32 hashvalue, int* bucketno, int* batchno); extern bool ExecScanHashBucket(HashJoinState* hjstate, ExprContext* econtext); +extern bool ExecParallelScanHashBucket(HashJoinState* hjstate, ExprContext* econtext); extern void ExecPrepHashTableForUnmatched(HashJoinState* hjstate); extern bool ExecScanHashTableForUnmatched(HashJoinState* hjstate, ExprContext* econtext); extern void ExecHashTableReset(HashJoinTable hashtable); extern void ExecHashTableResetMatchFlags(HashJoinTable hashtable); -extern void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, int* numbuckets, int* numbatches, - int* num_skew_mcvs, int4 localWorkMem, bool vectorized = false, OpMemInfo* memInfo = NULL); +extern void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, bool try_combined_work_mem, + int parallel_workers, size_t* space_allowed, int* numbuckets, int* numbatches, int* num_skew_mcvs, + int4 localWorkMem, bool vectorized = false, OpMemInfo* memInfo = NULL); extern double ExecChooseHashTableMaxTuples(int tupwidth, bool useskew, bool vectorized, double hash_table_bytes); extern int ExecHashGetSkewBucket(HashJoinTable hashtable, uint32 hashvalue); extern void ExecHashTableStats(HashJoinTable hashtable, int planid); @@ -48,4 +61,9 @@ extern int64 ExecSonicHashGetAtomArrayBytes( double ntuples, int m_arrSize, int m_atomSize, int64 atomTypeSize, bool hasNullFlag); extern uint8 EstimateBucketTypeSize(int nbuckets); +extern void ExecHashInitializeDSM(HashState* node, ParallelContext* pcxt, int nodeid); +extern void ExecHashInitializeWorker(HashState* node, void* pwcxt); +extern void ExecHashRetrieveInstrumentation(HashState* node); +extern void ExecShutdownHash(HashState* node); +extern void ExecHashGetInstrumentation(Instrumentation* instrument, HashJoinTable hashtable); #endif /* NODEHASH_H */ diff --git a/src/include/executor/nodeHashjoin.h b/src/include/executor/nodeHashjoin.h index 0a4eae26f..c2f608197 100644 --- a/src/include/executor/nodeHashjoin.h +++ b/src/include/executor/nodeHashjoin.h @@ -14,6 +14,7 @@ #ifndef NODEHASHJOIN_H #define NODEHASHJOIN_H +#include "access/parallel.h" #include "nodes/execnodes.h" #include "storage/buffile.h" @@ -25,4 +26,9 @@ extern void ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue, BufFile* extern void ExecEarlyFreeHashJoin(HashJoinState* node); extern void ExecReSetHashJoin(HashJoinState* node); +extern void ExecShutdownHashJoin(HashJoinState* node); +extern void ExecHashJoinInitializeDSM(HashJoinState* state, ParallelContext* pcxt, int nodeid); +extern void ExecHashJoinReInitializeDSM(HashJoinState* state, ParallelContext* pcxt); +extern void ExecHashJoinInitializeWorker(HashJoinState* state, void* pwcxt); + #endif /* NODEHASHJOIN_H */ diff --git a/src/include/executor/nodeIndexscan.h b/src/include/executor/nodeIndexscan.h index a2ecbe0fb..04199ff8b 100644 --- a/src/include/executor/nodeIndexscan.h +++ b/src/include/executor/nodeIndexscan.h @@ -25,6 +25,7 @@ extern void ExecIndexRestrPos(IndexScanState* node); extern void ExecReScanIndexScan(IndexScanState* node); extern void ExecIndexScanEstimate(IndexScanState *node, ParallelContext *pcxt); extern void ExecIndexScanInitializeDSM(IndexScanState *node, ParallelContext *pcxt, int nodeid); +extern void ExecIndexScanReInitializeDSM(IndexScanState *node, ParallelContext *pcxt); extern void ExecIndexScanInitializeWorker(IndexScanState *node, void *context); /* diff --git a/src/include/executor/nodeSeqscan.h b/src/include/executor/nodeSeqscan.h index 2860fdb88..b249e796a 100644 --- a/src/include/executor/nodeSeqscan.h +++ b/src/include/executor/nodeSeqscan.h @@ -29,6 +29,7 @@ extern void InitScanRelation(SeqScanState* node, EState* estate); /* parallel scan support */ extern void ExecSeqScanEstimate(SeqScanState *node, ParallelContext *pcxt); extern void ExecSeqScanInitializeDSM(SeqScanState *node, ParallelContext *pcxt, int nodeid); +extern void ExecSeqScanReInitializeDSM(SeqScanState *node, ParallelContext *pcxt); extern void ExecSeqScanInitializeWorker(SeqScanState *node, void *context); #endif /* NODESEQSCAN_H */ diff --git a/src/include/knl/knl_guc/knl_session_attr_sql.h b/src/include/knl/knl_guc/knl_session_attr_sql.h index 82a431bef..976588a74 100644 --- a/src/include/knl/knl_guc/knl_session_attr_sql.h +++ b/src/include/knl/knl_guc/knl_session_attr_sql.h @@ -202,8 +202,10 @@ typedef struct knl_session_attr_sql { double table_skewness_warning_threshold; bool enable_opfusion; bool enable_beta_opfusion; + bool enable_partition_opfusion; bool enable_beta_nestloop_fusion; bool parallel_leader_participation; + bool enable_parallel_hash; int opfusion_debug_mode; int single_shard_stmt; int force_parallel_mode; diff --git a/src/include/knl/knl_session.h b/src/include/knl/knl_session.h index 76fe7652a..6860d4694 100644 --- a/src/include/knl/knl_session.h +++ b/src/include/knl/knl_session.h @@ -2039,22 +2039,28 @@ typedef struct ParallelAppendState ParallelAppendState; /* Info need to pass from leader to worker */ struct ParallelHeapScanDescData; struct ParallelIndexScanDescData; +struct ParallelHashJoinState; +struct SharedHashInfo; typedef uint64 XLogRecPtr; typedef struct ParallelQueryInfo { - struct SharedExecutorInstrumentation *instrumentation; - BufferUsage *bufUsage; - char *tupleQueue; - char *pstmt_space; - char *param_space; + struct SharedExecutorInstrumentation* instrumentation; + BufferUsage* bufUsage; + char* tupleQueue; + char* pstmt_space; + char* param_space; Size param_len; int64 tuples_needed; /* tuple bound, see ExecSetTupleBound */ int eflags; int pscan_num; - ParallelHeapScanDescData **pscan; - int pappend_num; - ParallelAppendState **pappend; + ParallelHeapScanDescData** pscan; int piscan_num; - ParallelIndexScanDescData **piscan; + ParallelIndexScanDescData** piscan; + int pappend_num; + ParallelAppendState** pappend; + int jstate_num; + ParallelHashJoinState** jstate; + int hash_num; + SharedHashInfo** shared_info; } ParallelQueryInfo; struct BTShared; diff --git a/src/include/knl/knl_thread.h b/src/include/knl/knl_thread.h index cbb680e49..8851b11c8 100644 --- a/src/include/knl/knl_thread.h +++ b/src/include/knl/knl_thread.h @@ -2766,6 +2766,7 @@ typedef struct knl_t_bgworker_context { BufferUsage *save_pgBufferUsage; MemoryContext hpm_context; + MemoryContext memCxt; } knl_t_bgworker_context; struct shm_mq; diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h index 645ab08a9..b3d81378e 100644 --- a/src/include/nodes/execnodes.h +++ b/src/include/nodes/execnodes.h @@ -24,6 +24,7 @@ #include "storage/pagecompress.h" #include "utils/bloom_filter.h" #include "utils/reltrigger.h" +#include "utils/sharedtuplestore.h" #include "utils/sortsupport.h" #include "utils/tuplesort.h" #include "utils/tuplestore.h" @@ -701,6 +702,8 @@ typedef HASH_SEQ_STATUS TupleHashIterator; */ typedef struct ExprState ExprState; +struct ParallelHashJoinState; + typedef Datum (*ExprStateEvalFunc)(ExprState* expression, ExprContext* econtext, bool* isNull, ExprDoneCond* isDone); typedef ScalarVector* (*VectorExprFun)( ExprState* expression, ExprContext* econtext, bool* selVector, ScalarVector* inputVector, ExprDoneCond* isDone); @@ -2108,6 +2111,7 @@ typedef struct HashJoinState { bool hj_OuterNotEmpty; bool hj_streamBothSides; bool hj_rebuildHashtable; + bool isParallel; } HashJoinState; /* ---------------------------------------------------------------- @@ -2330,6 +2334,16 @@ typedef struct GatherState { int64 tuples_needed; } GatherState; +/* ---------------- + * Shared memory container for per-worker hash information + * ---------------- + */ +typedef struct SharedHashInfo { + int num_workers; + int plan_node_id; /* used to identify speicific plan */ + Instrumentation instrument[FLEXIBLE_ARRAY_MEMBER]; +} SharedHashInfo; + /* ---------------- * HashState information * ---------------- @@ -2341,7 +2355,12 @@ typedef struct HashState { int32 local_work_mem; /* work_mem local for this hash join */ int64 spill_size; + /* Parallel hash state. */ + struct ParallelHashJoinState* parallel_state; /* hashkeys is same as parent's hj_InnerHashKeys */ + + SharedHashInfo* shared_info; /* one entry per worker */ + Instrumentation* instrument; /* this worker's entry */ } HashState; /* ---------------- diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h index b145fb7da..7bac57497 100755 --- a/src/include/nodes/plannodes.h +++ b/src/include/nodes/plannodes.h @@ -1160,6 +1160,7 @@ typedef struct Unique { typedef struct Gather { Plan plan; int num_workers; + int rescan_param; /* ID of Param that signals a rescan, or -1 */ bool single_copy; } Gather; @@ -1177,9 +1178,10 @@ typedef struct Hash { Oid skewTable; /* outer join key's table OID, or InvalidOid */ AttrNumber skewColumn; /* outer join key's column #, or zero */ bool skewInherit; /* is outer join rel an inheritance tree? */ - Oid skewColType; /* datatype of the outer key column */ - int32 skewColTypmod; /* typmod of the outer key column */ - /* all other info is in the parent HashJoin node */ + double rows_total; /* estimate total rows if parallel_aware */ + Oid skewColType; /* datatype of the outer key column */ + int32 skewColTypmod; /* typmod of the outer key column */ + /* all other info is in the parent HashJoin node */ } Hash; /* ---------------- diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h index 109ebbfdc..9e902f00a 100755 --- a/src/include/nodes/relation.h +++ b/src/include/nodes/relation.h @@ -1274,9 +1274,10 @@ typedef struct MergePath { */ typedef struct HashPath { JoinPath jpath; - List* path_hashclauses; /* join clauses used for hashing */ - int num_batches; /* number of batches expected */ - OpMemInfo mem_info; /* Mem info for hash table */ + List* path_hashclauses; /* join clauses used for hashing */ + int num_batches; /* number of batches expected */ + double inner_rows_total; /* total inner rows expected */ + OpMemInfo mem_info; /* Mem info for hash table */ } HashPath; #ifdef PGXC @@ -1929,6 +1930,7 @@ typedef struct JoinCostWorkspace { /* private for cost_hashjoin code */ int numbuckets; int numbatches; + double inner_rows_total; /* Meminfo for joins */ OpMemInfo outer_mem_info; diff --git a/src/include/opfusion/opfusion_scan.h b/src/include/opfusion/opfusion_scan.h index be243de8f..97f117264 100644 --- a/src/include/opfusion/opfusion_scan.h +++ b/src/include/opfusion/opfusion_scan.h @@ -68,6 +68,10 @@ public: Relation m_rel; + Relation m_parentRel; + + Partition m_partRel; + TupleTableSlot* m_reslot; TupleDesc m_tupDesc; diff --git a/src/include/opfusion/opfusion_util.h b/src/include/opfusion/opfusion_util.h index 9ec71f022..145dd2c93 100644 --- a/src/include/opfusion/opfusion_util.h +++ b/src/include/opfusion/opfusion_util.h @@ -98,7 +98,14 @@ enum FusionType { NOBYPASS_JUST_VAR_FOR_AGGARGS, NOBYPASS_JUST_MERGE_UNSUPPORTED, - NOBYPASS_JUST_VAR_ALLOWED_IN_SORT + NOBYPASS_JUST_VAR_ALLOWED_IN_SORT, + + NOBYPASS_ZERO_PARTITION, + NOBYPASS_MULTI_PARTITION, + NOBYPASS_EXP_NOT_SUPPORT_IN_PARTITION, + NO_BYPASS_PARTITIONKEY_IS_NULL, + NOBYPASS_NO_UPDATE_PARTITIONKEY, + NOBYPASS_NO_INCLUDING_PARTITIONKEY }; enum FusionDebug { diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h index 487f23586..250c1217e 100755 --- a/src/include/optimizer/cost.h +++ b/src/include/optimizer/cost.h @@ -127,7 +127,8 @@ extern void initial_cost_mergejoin(PlannerInfo* root, JoinCostWorkspace* workspa extern void final_cost_mergejoin( PlannerInfo* root, MergePath* path, JoinCostWorkspace* workspace, SpecialJoinInfo* sjinfo, bool hasalternative); extern void initial_cost_hashjoin(PlannerInfo* root, JoinCostWorkspace* workspace, JoinType jointype, List* hashclauses, - Path* outer_path, Path* inner_path, SpecialJoinInfo* sjinfo, SemiAntiJoinFactors* semifactors, int dop); + Path* outer_path, Path* inner_path, SpecialJoinInfo* sjinfo, SemiAntiJoinFactors* semifactors, int dop, + bool parallel_hash); extern void final_cost_hashjoin(PlannerInfo* root, HashPath* path, JoinCostWorkspace* workspace, SpecialJoinInfo* sjinfo, SemiAntiJoinFactors* semifactors, bool hasalternative, int dop); extern void cost_rescan(PlannerInfo* root, Path* path, Cost* rescan_startup_cost, /* output parameters */ diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h index 8feb4603d..97bc0d754 100755 --- a/src/include/optimizer/pathnode.h +++ b/src/include/optimizer/pathnode.h @@ -99,7 +99,8 @@ extern MergePath* create_mergejoin_path(PlannerInfo* root, RelOptInfo* joinrel, extern HashPath* create_hashjoin_path(PlannerInfo* root, RelOptInfo* joinrel, JoinType jointype, JoinCostWorkspace* workspace, SpecialJoinInfo* sjinfo, SemiAntiJoinFactors* semifactors, Path* outer_path, - Path* inner_path, List* restrict_clauses, Relids required_outer, List* hashclauses, int dop = 1); + Path* inner_path, bool parallel_hash, List* restrict_clauses, Relids required_outer, List* hashclauses, + int dop = 1); extern Path* reparameterize_path(PlannerInfo* root, Path* path, Relids required_outer, double loop_count); diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h index 65c174503..47854170a 100755 --- a/src/include/storage/lwlock.h +++ b/src/include/storage/lwlock.h @@ -145,6 +145,7 @@ enum BuiltinTrancheIds { LWTRANCHE_PLDEBUG, // For Pldebugger LWTRANCHE_SHARED_TUPLESTORE, LWTRANCHE_PARALLEL_APPEND, + LWTRANCHE_PARALLEL_HASH_JOIN, /* * Each trancheId above should have a corresponding item in BuiltinTrancheNames; */ diff --git a/src/include/storage/pg_barrier.h b/src/include/storage/pg_barrier.h new file mode 100644 index 000000000..7808b9fba --- /dev/null +++ b/src/include/storage/pg_barrier.h @@ -0,0 +1,53 @@ +/* ------------------------------------------------------------------------- + * + * pg_barrier.h + * Barriers for synchronizing cooperating processes. + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * Portions Copyright (c) 2020 Huawei Technologies Co.,Ltd. + * + * src/include/storage/pg_barrier.h + * + * ------------------------------------------------------------------------- + */ +#ifndef PG_BARRIER_H +#define PG_BARRIER_H + +/* + * For the header previously known as "barrier.h", please include + * "port/atomics.h", which deals with atomics, compiler barriers and memory + * barriers. + */ + +#include "storage/spin.h" +#include "miscadmin.h" +typedef struct { + pthread_mutex_t mutex; + pthread_cond_t cond; +} ConditionVariable; + +typedef struct Barrier { + slock_t mutex; + int phase; /* phase counter */ + int participants; /* the number of participants attached */ + int arrived; /* the number of participants that have + * arrived */ + int elected; /* highest phase elected */ + bool static_party; /* used only for assertions */ + ConditionVariable condition_variable; +} Barrier; + +extern void BarrierInit(Barrier* barrier, int num_workers); +extern bool BarrierArriveAndWaitStatus(Barrier* barrier); +extern bool BarrierArriveAndDetach(Barrier* barrier); +extern int BarrierAttach(Barrier* barrier); +extern bool BarrierDetach(Barrier* barrier); +extern int BarrierPhase(Barrier* barrier); +extern int BarrierParticipants(Barrier* barrier); + +extern void ConditionVariableInit(ConditionVariable*); +extern void ConditionVariableBroadcast(ConditionVariable*); + +#define BarrierArriveAndWait(a, b) BarrierArriveAndWaitStatus(a) +#endif /* PG_BARRIER_H */ diff --git a/src/include/utils/dsa.h b/src/include/utils/dsa.h new file mode 100644 index 000000000..7e4b3b232 --- /dev/null +++ b/src/include/utils/dsa.h @@ -0,0 +1,82 @@ +/* ------------------------------------------------------------------------- + * + * dsa.h + * Dynamic shared memory areas. + * + * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * Portions Copyright (c) 2020 Huawei Technologies Co.,Ltd. + * + * IDENTIFICATION + * src/include/utils/dsa.h + * + * ------------------------------------------------------------------------- + */ +#ifndef DSA_H +#define DSA_H + +#include "utils/atomic.h" + +/* + * If this system only uses a 32-bit value for Size, then use the 32-bit + * implementation of DSA. This limits the amount of DSA that can be created + * to something significantly less than the entire 4GB address space because + * the DSA pointer must encode both a segment identifier and an offset, but + * that shouldn't be a significant limitation in practice. + * + * If this system doesn't support atomic operations on 64-bit values, then + * we fall back to 32-bit dsa_pointer for lack of other options. + * + * For testing purposes, USE_SMALL_DSA_POINTER can be defined to force the use + * of 32-bit dsa_pointer even on systems capable of supporting a 64-bit + * dsa_pointer. + */ +#if SIZEOF_SIZE_T == 4 +#define SIZEOF_DSA_POINTER 4 +#else +#define SIZEOF_DSA_POINTER 8 +#endif + +/* + * The type of 'relative pointers' to memory allocated by a dynamic shared + * area. dsa_pointer values can be shared with other processes, but must be + * converted to backend-local pointers before they can be dereferenced. See + * dsa_get_address. Also, an atomic version and appropriately sized atomic + * operations. + */ +#if SIZEOF_DSA_POINTER == 4 +typedef uint32 dsa_pointer; +typedef pg_atomic_uint32 dsa_pointer_atomic; +#define dsa_pointer_atomic_init pg_atomic_init_u32 +#define dsa_pointer_atomic_read pg_atomic_read_u32 +#define dsa_pointer_atomic_write pg_atomic_write_u32 +#define dsa_pointer_atomic_fetch_add pg_atomic_fetch_add_u32 +#define dsa_pointer_atomic_compare_exchange pg_atomic_compare_exchange_u32 +#define DSA_POINTER_FORMAT "%08x" +#else +typedef uint64 dsa_pointer; +typedef pg_atomic_uint64 dsa_pointer_atomic; +#define dsa_pointer_atomic_init pg_atomic_init_u64 +#define dsa_pointer_atomic_read pg_atomic_read_u64 +#define dsa_pointer_atomic_write pg_atomic_write_u64 +#define dsa_pointer_atomic_fetch_add pg_atomic_fetch_add_u64 +#define dsa_pointer_atomic_compare_exchange pg_atomic_compare_exchange_u64 +#define DSA_POINTER_FORMAT "%016" INT64_MODIFIER "x" +#endif + +/* A sentinel value for dsa_pointer used to indicate failure to allocate. */ +#define InvalidDsaPointer (0) + +/* Check if a dsa_pointer value is valid. */ +#define DsaPointerIsValid(x) ((x) != InvalidDsaPointer) + +/* Allocate uninitialized memory with error on out-of-memory. */ +#define dsa_allocate(ctx, size) MemoryContextAlloc(ctx, size) + +/* Allocate zero-initialized memory with error on out-of-memory. */ +#define dsa_allocate0(ctx, size) MemoryContextAllocZero(ctx, size) + +#define dsa_free(ctx, dp) pfree(dp) +#define dsa_get_address(a, b) (b) + +#endif /* DSA_H */ diff --git a/src/include/utils/partitionmap.h b/src/include/utils/partitionmap.h index 26291665d..81d6bc30b 100755 --- a/src/include/utils/partitionmap.h +++ b/src/include/utils/partitionmap.h @@ -93,6 +93,7 @@ extern void RelationInitPartitionMap(Relation relation); extern int partOidGetPartSequence(Relation rel, Oid partOid); extern Oid getRangePartitionOid(Relation relation, Const** partKeyValue, int* partIndex, bool topClosed); +extern Oid getPartitionOidByParam(Relation relation, Param *paramArg, ParamExternData *prm); extern List* getPartitionBoundaryList(Relation rel, int sequence); extern Oid partitionKeyValueListGetPartitionOid(Relation rel, List* partKeyValueList, bool topClosed); diff --git a/src/include/utils/partitionmap_gs.h b/src/include/utils/partitionmap_gs.h index 5e2d0b0ae..dd7881c73 100755 --- a/src/include/utils/partitionmap_gs.h +++ b/src/include/utils/partitionmap_gs.h @@ -345,6 +345,7 @@ typedef struct PruningResult { /*if interval partitions is empty, intervalOffset=-1*/ Bitmapset* intervalSelectedPartitions; List* ls_rangeSelectedPartitions; + Param* paramArg; } PruningResult; extern Oid partIDGetPartOid(Relation relation, PartitionIdentifier* partID); @@ -363,5 +364,6 @@ extern Oid GetNeedDegradToRangePartOid(Relation rel, Oid partOid); extern RangeElement* CopyRangeElementsWithoutBoundary(const RangeElement* src, int elementNum); extern char* ReadIntervalStr(HeapTuple tuple, TupleDesc tupleDesc); extern oidvector* ReadIntervalTablespace(HeapTuple tuple, TupleDesc tupleDesc); +extern void get_typlenbyval(Oid typid, int16 *typlen, bool *typbyval); int ValueCmpLowBoudary(Const** partKeyValue, const RangeElement* partition, Interval* intervalValue); #endif /* PARTITIONMAP_GS_H_ */ diff --git a/src/include/utils/sharedtuplestore.h b/src/include/utils/sharedtuplestore.h index 14c6e6b06..de74d04fc 100644 --- a/src/include/utils/sharedtuplestore.h +++ b/src/include/utils/sharedtuplestore.h @@ -44,7 +44,7 @@ extern void sts_begin_parallel_scan(SharedTuplestoreAccessor* accessor); extern void sts_end_parallel_scan(SharedTuplestoreAccessor* accessor); -extern void sts_puttuple(SharedTuplestoreAccessor* accessor, void* meta_data, MinimalTuple tuple); +extern void sts_puttuple(SharedTuplestoreAccessor* accessor, const void* meta_data, MinimalTuple tuple); extern MinimalTuple sts_parallel_scan_next(SharedTuplestoreAccessor* accessor, void* meta_data); diff --git a/src/test/regress/expected/llt_coverage_atc.out b/src/test/regress/expected/llt_coverage_atc.out index c7c1f8418..468ea4420 100644 --- a/src/test/regress/expected/llt_coverage_atc.out +++ b/src/test/regress/expected/llt_coverage_atc.out @@ -176,8 +176,8 @@ set enable_hashjoin=on; set enable_nestloop=off; set enable_mergejoin=off; explain (ANALYZE on, BUFFERS on, TIMING on, detail off, format text) select * from llt_show_datanode_buffers, llt_show_datanode_buffers_1 where llt_show_datanode_buffers.q2=llt_show_datanode_buffers_1.q1; ---? .*QUERY PLAN ---? .* +--?.* +--?.* --? Hash Join \(cost=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=[0-9]* width=[0-9]*\) \(actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=[0-9]* loops=[0-9]*\) Hash Cond: (llt_show_datanode_buffers_1.q1 = llt_show_datanode_buffers.q2) --? (Buffers: shared hit=.* dirtied=.*) @@ -450,8 +450,8 @@ explain (ANALYZE on, BUFFERS on, TIMING off, detail off, format JSON, cpu on) se (1 row) explain (ANALYZE on, BUFFERS on, TIMING on, detail on, format text) select * from llt_show_datanode_buffers, llt_show_datanode_buffers_1 where llt_show_datanode_buffers.q2=llt_show_datanode_buffers_1.q1; ---? .*QUERY PLAN ---? .* +--?.* +--?.* --? Hash Join \(cost=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=[0-9]* width=[0-9]*\) \(actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=[0-9]* loops=[0-9]*\) Hash Cond: (llt_show_datanode_buffers_1.q1 = llt_show_datanode_buffers.q2) --? (Buffers: shared hit=.* dirtied=.*) diff --git a/src/test/regress/expected/parallel_append.out b/src/test/regress/expected/parallel_append.out index 76fa27be1..c77521bd1 100644 --- a/src/test/regress/expected/parallel_append.out +++ b/src/test/regress/expected/parallel_append.out @@ -26,126 +26,122 @@ set enable_parallel_append to on; ------------------------------------------- -- 1. union && union all ------------------------------------------- -explain select * from a union select * from b; ---?.*QUERY PLAN.* ---?-------------.* ---? HashAggregate (cost=.* rows=10 width=12) +explain (costs off) select * from a union select * from b; + QUERY PLAN +------------------------------------------ + HashAggregate Group By Key: a.a1, a.a2, a.a3 ---? -> Append (cost=.* rows=10 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Append + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Parallel Seq Scan on a + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) + -> Parallel Seq Scan on b (9 rows) -explain select * from a union all select * from b; ---?.*QUERY PLAN.* ---?-----------.* ---? Gather (cost=.* rows=10 width=12) +explain (costs off) select * from a union all select * from b; + QUERY PLAN +------------------------------------ + Gather Number of Workers: 2 ---? -> Parallel Append (cost=.* rows=4 width=12) ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) + -> Parallel Append + -> Parallel Seq Scan on a + -> Parallel Seq Scan on b (5 rows) -explain select * from a where a1 > 4 union select * from b where b1 < 6; ---?.*QUERY PLAN.* ---?---------.* ---? HashAggregate (cost=.* rows=3 width=12) +explain (costs off) select * from a where a1 > 4 union select * from b where b1 < 6; + QUERY PLAN +------------------------------------------ + HashAggregate Group By Key: a.a1, a.a2, a.a3 ---? -> Append (cost=.* rows=3 width=12) ---? -> Gather (cost=.* rows=1 width=12) + -> Append + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on a (cost=.* rows=1 width=12) + -> Parallel Seq Scan on a Filter: (a1 > 4) ---? -> Gather (cost=.* rows=2 width=12) + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on b (cost=.* rows=1 width=12) + -> Parallel Seq Scan on b Filter: (b1 < 6) (11 rows) -explain select * from a where a1 > 4 union all select * from b where b1 < 6; ---?.*QUERY PLAN.* ---?--------.* ---? Result (cost=.* rows=3 width=12) ---? -> Append (cost=.* rows=3 width=12) ---? -> Gather (cost=.* rows=1 width=12) +explain (costs off) select * from a where a1 > 4 union all select * from b where b1 < 6; + QUERY PLAN +------------------------------------------ + Result + -> Append + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on a (cost=.* rows=1 width=12) + -> Parallel Seq Scan on a Filter: (a1 > 4) ---? -> Gather (cost=.* rows=2 width=12) + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on b (cost=.* rows=1 width=12) + -> Parallel Seq Scan on b Filter: (b1 < 6) (10 rows) -explain select * from c where c1 in (select a1 from a union select b1 from b); ---?.*QUERY PLAN.* ---?-------.* ---? Hash Join (cost=.* rows=5 width=12) +explain (costs off) select * from c where c1 in (select a1 from a union select b1 from b); + QUERY PLAN +--------------------------------- + Hash Join Hash Cond: (a.a1 = c.c1) ---? -> HashAggregate (cost=.* rows=10 width=4) + -> HashAggregate Group By Key: a.a1 ---? -> Append (cost=.* rows=10 width=4) ---? -> Seq Scan on a (cost=.* rows=5 width=4) ---? -> Seq Scan on b (cost=.* rows=5 width=4) ---? -> Hash (cost=.* rows=5 width=12) ---? -> Seq Scan on c (cost=.* rows=5 width=12) + -> Append + -> Seq Scan on a + -> Seq Scan on b + -> Hash + -> Seq Scan on c (9 rows) -explain select * from (select * from a union all select * from b) as ta, c where ta.a1 = c.c1; ---?.*QUERY PLAN.* ---?------.* ---? Hash Join (cost=.* rows=5 width=24) - Hash Cond: (a.a1 = c.c1) ---? -> Gather (cost=.* rows=10 width=12) - Number of Workers: 2 ---? -> Parallel Append (cost=.* rows=4 width=12) ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) ---? -> Hash (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) - Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=3 width=12) +explain (costs off) select * from (select * from a union all select * from b) as ta, c where ta.a1 = c.c1; + QUERY PLAN +------------------------------------------ + Gather + Number of Workers: 2 + -> Parallel Hash Join + Hash Cond: (a.a1 = c.c1) + -> Parallel Append + -> Parallel Seq Scan on a + -> Parallel Seq Scan on b + -> Parallel Hash + -> Parallel Seq Scan on c +(9 rows) + +explain (costs off) select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1 order by 1,2,3,4,5,6; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: d.d1, d.d2, d.d3, a.a1, a.a2, a.a3 + -> Gather + Number of Workers: 1 + -> Parallel Hash Left Join + Hash Cond: (d.d1 = a.a1) + -> Parallel Seq Scan on d + -> Parallel Hash + -> Parallel Append + -> Parallel Seq Scan on a + -> Parallel Seq Scan on b (11 rows) -explain select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1; ---?.*QUERY PLAN.* ---?-----.* ---? Hash Right Join (cost=.* rows=10 width=24) - Hash Cond: (a.a1 = d.d1) ---? -> Gather (cost=.* rows=10 width=12) - Number of Workers: 2 ---? -> Parallel Append (cost=.* rows=4 width=12) ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) ---? -> Hash (cost=.* rows=10 width=12) ---? -> Gather (cost=.* rows=10 width=12) - Number of Workers: 1 ---? -> Parallel Seq Scan on d (cost=.* rows=6 width=12) -(11 rows) - -explain select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from b) t, d where t.a1=d1 group by d.d1 order by 1,2; ---?.*QUERY PLAN.* ---?-------.* ---? Sort (cost=.* rows=5 width=28) +explain (costs off) select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from b) t, d where t.a1=d1 group by d.d1 order by 1,2; + QUERY PLAN +------------------------------------------------------------ + Sort Sort Key: d.d1, (sum(d.d2)) ---? -> HashAggregate (cost=.* rows=5 width=28) + -> HashAggregate Group By Key: d.d1 ---? -> Hash Join (cost=.* rows=10 width=12) - Hash Cond: (a.a1 = d.d1) ---? -> Gather (cost=.* rows=10 width=8) - Number of Workers: 2 ---? -> Parallel Append (cost=.* rows=4 width=8) ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=8) ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=8) ---? -> Hash (cost=.* rows=10 width=8) ---? -> Gather (cost=.* rows=10 width=8) - Number of Workers: 1 ---? -> Parallel Seq Scan on d (cost=.* rows=6 width=8) -(15 rows) + -> Gather + Number of Workers: 1 + -> Parallel Hash Join + Hash Cond: (d.d1 = a.a1) + -> Parallel Seq Scan on d + -> Parallel Hash + -> Parallel Append + -> Parallel Seq Scan on a + -> Parallel Seq Scan on b +(13 rows) select * from a union select * from b; a1 | a2 | a3 @@ -204,23 +200,23 @@ select * from (select * from a union all select * from b) as ta, c where ta.a1 = 8 | 8 | 8 | 8 | 8 | 8 (2 rows) -select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1; +select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1 order by 1,2,3,4,5,6; d1 | d2 | d3 | a1 | a2 | a3 ----+----+----+----+----+---- - 4 | 6 | 8 | 4 | 4 | 4 - 4 | 5 | 6 | 4 | 4 | 4 - 5 | 7 | 10 | 5 | 5 | 5 - 5 | 6 | 7 | 5 | 5 | 5 - 1 | 3 | 2 | 1 | 1 | 1 1 | 2 | 3 | 1 | 1 | 1 - 2 | 4 | 4 | 2 | 2 | 2 + 1 | 3 | 2 | 1 | 1 | 1 2 | 3 | 4 | 2 | 2 | 2 - 3 | 5 | 6 | 3 | 3 | 3 + 2 | 4 | 4 | 2 | 2 | 2 3 | 4 | 5 | 3 | 3 | 3 - 4 | 6 | 8 | 4 | 4 | 4 + 3 | 5 | 6 | 3 | 3 | 3 4 | 5 | 6 | 4 | 4 | 4 - 5 | 7 | 10 | 5 | 5 | 5 + 4 | 5 | 6 | 4 | 4 | 4 + 4 | 6 | 8 | 4 | 4 | 4 + 4 | 6 | 8 | 4 | 4 | 4 5 | 6 | 7 | 5 | 5 | 5 + 5 | 6 | 7 | 5 | 5 | 5 + 5 | 7 | 10 | 5 | 5 | 5 + 5 | 7 | 10 | 5 | 5 | 5 (14 rows) select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from b) t, d where t.a1=d1 group by d.d1 order by 1,2; @@ -236,42 +232,38 @@ select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from -- set parallel_workers of table a to 0, subplan of seqscan on a should not be paralleled alter table a set (parallel_workers=0); explain (costs off) select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1 order by 1,2,3,4,5,6; - QUERY PLAN ------------------------------------------------- - Sort - Sort Key: d.d1, d.d2, d.d3, a.a1, a.a2, a.a3 - -> Hash Right Join - Hash Cond: (a.a1 = d.d1) - -> Gather - Number of Workers: 2 - -> Parallel Append - -> Seq Scan on a - -> Parallel Seq Scan on b - -> Hash - -> Gather - Number of Workers: 1 - -> Parallel Seq Scan on d -(13 rows) - -explain (costs off) select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from b) t, d where t.a1=d1 group by d.d1 order by 1,2; QUERY PLAN ------------------------------------------------------ + Sort + Sort Key: d.d1, d.d2, d.d3, a.a1, a.a2, a.a3 + -> Gather + Number of Workers: 1 + -> Parallel Hash Left Join + Hash Cond: (d.d1 = a.a1) + -> Parallel Seq Scan on d + -> Parallel Hash + -> Parallel Append + -> Seq Scan on a + -> Parallel Seq Scan on b +(11 rows) + +explain (costs off) select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from b) t, d where t.a1=d1 group by d.d1 order by 1,2; + QUERY PLAN +------------------------------------------------------------ Sort Sort Key: d.d1, (sum(d.d2)) -> HashAggregate Group By Key: d.d1 - -> Hash Join - Hash Cond: (a.a1 = d.d1) - -> Gather - Number of Workers: 2 - -> Parallel Append - -> Seq Scan on a - -> Parallel Seq Scan on b - -> Hash - -> Gather - Number of Workers: 1 - -> Parallel Seq Scan on d -(15 rows) + -> Gather + Number of Workers: 1 + -> Parallel Hash Join + Hash Cond: (d.d1 = a.a1) + -> Parallel Seq Scan on d + -> Parallel Hash + -> Parallel Append + -> Seq Scan on a + -> Parallel Seq Scan on b +(13 rows) select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1 order by 1,2,3,4,5,6; d1 | d2 | d3 | a1 | a2 | a3 @@ -321,35 +313,35 @@ select * from c except all select * from b where b1 >4; 9 | 9 | 9 (3 rows) -explain select * from c except select * from b where b1 >4; ---?.*QUERY PLAN.* ---?------.* ---? HashSetOp Except (cost=.* rows=5 width=12) ---? -> Append (cost=.* rows=10 width=12) ---? -> Subquery Scan on "*SELECT* 1" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) +explain (costs off) select * from c except select * from b where b1 >4; + QUERY PLAN +------------------------------------------------ + HashSetOp Except + -> Append + -> Subquery Scan on "*SELECT* 1" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=3 width=12) ---? -> Subquery Scan on "*SELECT* 2" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Parallel Seq Scan on c + -> Subquery Scan on "*SELECT* 2" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) + -> Parallel Seq Scan on b Filter: (b1 > 4) (11 rows) -explain select * from c except all select * from b where b1 >4; ---?.*QUERY PLAN.* ---?--------.* ---? HashSetOp Except All (cost=.* rows=5 width=12) ---? -> Append (cost=.* rows=10 width=12) ---? -> Subquery Scan on "*SELECT* 1" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) +explain (costs off) select * from c except all select * from b where b1 >4; + QUERY PLAN +------------------------------------------------ + HashSetOp Except All + -> Append + -> Subquery Scan on "*SELECT* 1" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=3 width=12) ---? -> Subquery Scan on "*SELECT* 2" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Parallel Seq Scan on c + -> Subquery Scan on "*SELECT* 2" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) + -> Parallel Seq Scan on b Filter: (b1 > 4) (11 rows) @@ -372,35 +364,35 @@ select * from e intersect all select * from c where c1 != 8; 0 | 0 | 0 (3 rows) -explain select * from e intersect select * from c; ---?.*QUERY PLAN.* ---?----------.* ---? HashSetOp Intersect (cost=.* rows=5 width=12) ---? -> Append (cost=.* rows=12 width=12) ---? -> Subquery Scan on "*SELECT* 2" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) +explain (costs off) select * from e intersect select * from c; + QUERY PLAN +------------------------------------------------ + HashSetOp Intersect + -> Append + -> Subquery Scan on "*SELECT* 2" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=3 width=12) ---? -> Subquery Scan on "*SELECT* 1" (cost=.* rows=7 width=12) ---? -> Gather (cost=.* rows=7 width=12) + -> Parallel Seq Scan on c + -> Subquery Scan on "*SELECT* 1" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on e (cost=.* rows=4 width=12) + -> Parallel Seq Scan on e (10 rows) -explain select * from e intersect all select * from c where c1 != 8; ---?.*QUERY PLAN.* ---?---------.* ---? HashSetOp Intersect All (cost=.* rows=4 width=12) ---? -> Append (cost=.* rows=11 width=12) ---? -> Subquery Scan on "*SELECT* 2" (cost=.* rows=4 width=12) ---? -> Gather (cost=.* rows=4 width=12) +explain (costs off) select * from e intersect all select * from c where c1 != 8; + QUERY PLAN +------------------------------------------------ + HashSetOp Intersect All + -> Append + -> Subquery Scan on "*SELECT* 2" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=2 width=12) + -> Parallel Seq Scan on c Filter: (c1 <> 8) ---? -> Subquery Scan on "*SELECT* 1" (cost=.* rows=7 width=12) ---? -> Gather (cost=.* rows=7 width=12) + -> Subquery Scan on "*SELECT* 1" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on e (cost=.* rows=4 width=12) + -> Parallel Seq Scan on e (11 rows) -------------------------------------- @@ -505,134 +497,132 @@ select * from (select * from a union all select * from b)as x, (select * from d 8 | 8 | 8 | 8 | 8 | 8 (20 rows) -explain select * from e intersect (select * from a except select * from b union select * from c); ---?.*QUERY PLAN.* ---?------.* ---? HashSetOp Intersect (cost=.* rows=7 width=12) ---? -> Append (cost=.* rows=17 width=12) ---? -> Subquery Scan on "*SELECT* 1" (cost=.* rows=7 width=12) ---? -> Gather (cost=.* rows=7 width=12) +explain (costs off) select * from e intersect (select * from a except select * from b union select * from c); + QUERY PLAN +------------------------------------------------------------------------------------- + HashSetOp Intersect + -> Append + -> Subquery Scan on "*SELECT* 1" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on e (cost=.* rows=4 width=12) ---? -> Result (cost=.* rows=10 width=12) ---? -> HashAggregate (cost=.* rows=10 width=12) + -> Parallel Seq Scan on e + -> Result + -> HashAggregate Group By Key: "*SELECT* 2".a1, "*SELECT* 2".a2, "*SELECT* 2".a3 ---? -> Append (cost=.* rows=10 width=12) ---? -> Result (cost=.* rows=5 width=12) ---? -> HashSetOp Except (cost=.* rows=5 width=12) ---? -> Append (cost=.* rows=10 width=12) ---? -> Subquery Scan on "*SELECT* 2" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Append + -> Result + -> HashSetOp Except + -> Append + -> Subquery Scan on "*SELECT* 2" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> Subquery Scan on "*SELECT* 3" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Parallel Seq Scan on a + -> Subquery Scan on "*SELECT* 3" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Parallel Seq Scan on b + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=3 width=12) + -> Parallel Seq Scan on c (24 rows) -explain select d2 from d except all (select d2 from d except select c1 from c) union all select e1 from e; ---?.*QUERY PLAN.* ---?-------------.* ---? Append (cost=.* rows=17 width=4) ---? -> Result (cost=.* rows=10 width=4) ---? -> HashSetOp Except All (cost=.* rows=10 width=4) ---? -> Append (cost=.* rows=16 width=4) ---? -> Subquery Scan on "*SELECT* 1" (cost=.* rows=10 width=4) ---? -> Gather (cost=.* rows=10 width=4) +explain (costs off) select d2 from d except all (select d2 from d except select c1 from c) union all select e1 from e; + QUERY PLAN +------------------------------------------------------------------------------ + Append + -> Result + -> HashSetOp Except All + -> Append + -> Subquery Scan on "*SELECT* 1" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on d (cost=.* rows=6 width=4) ---? -> Result (cost=.* rows=6 width=4) ---? -> HashSetOp Except (cost=.* rows=6 width=4) ---? -> Append (cost=.* rows=15 width=4) ---? -> Subquery Scan on "*SELECT* 2" (cost=.* rows=10 width=4) ---? -> Gather (cost=.* rows=10 width=4) + -> Parallel Seq Scan on d + -> Result + -> HashSetOp Except + -> Append + -> Subquery Scan on "*SELECT* 2" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on d (cost=.* rows=6 width=4) ---? -> Subquery Scan on "*SELECT* 3" (cost=.* rows=5 width=4) ---? -> Gather (cost=.* rows=5 width=4) + -> Parallel Seq Scan on d + -> Subquery Scan on "*SELECT* 3" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=3 width=4) ---? -> Gather (cost=.* rows=7 width=4) + -> Parallel Seq Scan on c + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on e (cost=.* rows=4 width=4) + -> Parallel Seq Scan on e (22 rows) -explain select * from a union all (select * from b union select * from c where c1 < 5); ---?.*QUERY PLAN.* ---?------------.* ---? Append (cost=.* rows=12 width=12) ---? -> Gather (cost=.* rows=5 width=12) +explain (costs off) select * from a union all (select * from b union select * from c where c1 < 5); + QUERY PLAN +------------------------------------------------ + Append + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> HashAggregate (cost=.* rows=7 width=12) + -> Parallel Seq Scan on a + -> HashAggregate Group By Key: b.b1, b.b2, b.b3 ---? -> Append (cost=.* rows=7 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Append + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) ---? -> Gather (cost=.* rows=2 width=12) + -> Parallel Seq Scan on b + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=1 width=12) + -> Parallel Seq Scan on c Filter: (c1 < 5) (14 rows) -explain select * from a except select * from b union select * from c; ---?.*QUERY PLAN.* ---?-------------.* ---? HashAggregate (cost=.* rows=10 width=12) +explain (costs off) select * from a except select * from b union select * from c; + QUERY PLAN +------------------------------------------------------------------- + HashAggregate Group By Key: "*SELECT* 1".a1, "*SELECT* 1".a2, "*SELECT* 1".a3 ---? -> Append (cost=.* rows=10 width=12) ---? -> Result (cost=.* rows=5 width=12) ---? -> HashSetOp Except (cost=.* rows=5 width=12) ---? -> Append (cost=.* rows=10 width=12) ---? -> Subquery Scan on "*SELECT* 1" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Append + -> Result + -> HashSetOp Except + -> Append + -> Subquery Scan on "*SELECT* 1" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> Subquery Scan on "*SELECT* 2" (cost=.* rows=5 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Parallel Seq Scan on a + -> Subquery Scan on "*SELECT* 2" + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) ---? -> Gather (cost=.* rows=5 width=12) + -> Parallel Seq Scan on b + -> Gather Number of Workers: 1 ---? -> Parallel Seq Scan on c (cost=.* rows=3 width=12) + -> Parallel Seq Scan on c (17 rows) -explain select * from b union all (select * from (select * from a union all select * from b)); ---?.* QUERY PLAN.* ---?---------.* ---? Gather (cost=.* rows=15 width=12) +explain (costs off) select * from b union all (select * from (select * from a union all select * from b)); + QUERY PLAN +------------------------------------ + Gather Number of Workers: 2 ---? -> Parallel Append (cost=.* rows=6 width=12) ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) + -> Parallel Append + -> Parallel Seq Scan on b + -> Parallel Seq Scan on a + -> Parallel Seq Scan on b (6 rows) -explain select * from (select * from a union all select * from b)as x, (select * from d union all select* from e)as y +explain (costs off) select * from (select * from a union all select * from b)as x, (select * from d union all select* from e)as y where x.a1 = y.d1 order by 1, 2, 3, 4, 5, 6; ---?.* QUERY PLAN.* ---?--------.* ---? Sort (cost=.* rows=10 width=24) + QUERY PLAN +------------------------------------------------------ + Sort Sort Key: a.a1, a.a2, a.a3, d.d2, d.d3 ---? -> Hash Join (cost=.* rows=10 width=24) - Hash Cond: (d.d1 = a.a1) ---? -> Gather (cost=.* rows=17 width=12) - Number of Workers: 2 ---? -> Parallel Append (cost=.* rows=7 width=12) ---? -> Parallel Seq Scan on d (cost=.* rows=6 width=12) ---? -> Parallel Seq Scan on e (cost=.* rows=4 width=12) ---? -> Hash (cost=.* rows=10 width=12) ---? -> Gather (cost=.* rows=10 width=12) - Number of Workers: 2 ---? -> Parallel Append (cost=.* rows=4 width=12) ---? -> Parallel Seq Scan on a (cost=.* rows=3 width=12) ---? -> Parallel Seq Scan on b (cost=.* rows=3 width=12) -(15 rows) + -> Gather + Number of Workers: 2 + -> Parallel Hash Join + Hash Cond: (d.d1 = a.a1) + -> Parallel Append + -> Parallel Seq Scan on d + -> Parallel Seq Scan on e + -> Parallel Hash + -> Parallel Append + -> Parallel Seq Scan on a + -> Parallel Seq Scan on b +(13 rows) ---------------------------------------- -- clean up diff --git a/src/test/regress/expected/parallel_hashjoin.out b/src/test/regress/expected/parallel_hashjoin.out index cb1809d8d..163aa7531 100644 --- a/src/test/regress/expected/parallel_hashjoin.out +++ b/src/test/regress/expected/parallel_hashjoin.out @@ -43,12 +43,12 @@ explain (costs off) select * from parallel_hashjoin_test_a left outer join paral Sort Key: parallel_hashjoin_test_a.id -> Gather Number of Workers: 2 - -> Hash Left Join + -> Parallel Hash Left Join Hash Cond: (parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id) -> Parallel Seq Scan on parallel_hashjoin_test_a Filter: (id < 10) - -> Hash - -> Seq Scan on parallel_hashjoin_test_b + -> Parallel Hash + -> Parallel Seq Scan on parallel_hashjoin_test_b (10 rows) select * from parallel_hashjoin_test_a left outer join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id where parallel_hashjoin_test_a.id < 10 order by parallel_hashjoin_test_a.id; @@ -66,19 +66,20 @@ select * from parallel_hashjoin_test_a left outer join parallel_hashjoin_test_b (9 rows) -- Forbid parallel Hash Right Join or Hash Full Join. -explain (costs off)select * from parallel_hashjoin_test_a right outer join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id order by parallel_hashjoin_test_a.id; - QUERY PLAN --------------------------------------------------------------------------------- - Sort - Sort Key: parallel_hashjoin_test_a.id - -> Hash Right Join - Hash Cond: (parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id) - -> Seq Scan on parallel_hashjoin_test_a - -> Hash - -> Seq Scan on parallel_hashjoin_test_b -(7 rows) +explain (costs off)select * from parallel_hashjoin_test_a full join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id order by parallel_hashjoin_test_a.id limit 10; + QUERY PLAN +-------------------------------------------------------------------------------------- + Limit + -> Sort + Sort Key: parallel_hashjoin_test_a.id + -> Hash Full Join + Hash Cond: (parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id) + -> Seq Scan on parallel_hashjoin_test_a + -> Hash + -> Seq Scan on parallel_hashjoin_test_b +(8 rows) -select * from parallel_hashjoin_test_a right outer join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id order by parallel_hashjoin_test_a.id; +select * from parallel_hashjoin_test_a full join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id order by parallel_hashjoin_test_a.id limit 10; id | id ----+---- 1 | 1 @@ -93,6 +94,8 @@ select * from parallel_hashjoin_test_a right outer join parallel_hashjoin_test_b 10 | 10 (10 rows) +drop table parallel_hashjoin_test_a; +drop table parallel_hashjoin_test_b; reset parallel_setup_cost; reset min_parallel_table_scan_size; reset parallel_tuple_cost; diff --git a/src/test/regress/expected/parallel_nested_loop.out b/src/test/regress/expected/parallel_nested_loop.out index e8d7b8bb4..b5309f876 100644 --- a/src/test/regress/expected/parallel_nested_loop.out +++ b/src/test/regress/expected/parallel_nested_loop.out @@ -26,6 +26,7 @@ select * from parallel_nested_loop_test_a left outer join parallel_nested_loop_t set parallel_setup_cost = 0; set min_parallel_table_scan_size=0; set force_parallel_mode=on; +set parallel_tuple_cost= 0.1; explain (costs off) select * from parallel_nested_loop_test_a left outer join parallel_nested_loop_test_b on parallel_nested_loop_test_a.id = 1; QUERY PLAN -------------------------------------------------------------- @@ -53,3 +54,4 @@ drop table parallel_nested_loop_test_b; reset parallel_setup_cost; reset min_parallel_table_scan_size; reset force_parallel_mode; +reset parallel_tuple_cost; diff --git a/src/test/regress/expected/parallel_query.out b/src/test/regress/expected/parallel_query.out index a0f05d0cd..b8075a965 100644 --- a/src/test/regress/expected/parallel_query.out +++ b/src/test/regress/expected/parallel_query.out @@ -362,4 +362,52 @@ reset parallel_tuple_cost; reset max_parallel_workers_per_gather; reset min_parallel_table_scan_size; reset parallel_leader_participation; -reset min_parallel_index_scan_size; \ No newline at end of file +reset min_parallel_index_scan_size; +-- nestloop +set enable_hashjoin=off; +set enable_mergejoin=off; +explain (costs off, analyse on) select schemaname, tablename from pg_tables where tablename like 'sql%' order by tablename; +--?.* +--?.* +--? Nested Loop Left Join (actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=7 loops=1) + Join Filter: (n.oid = c.relnamespace) +--? Rows Removed by Join Filter: [0-9]* +--? -> Index Scan using pg_class_relname_nsp_index on pg_class c (actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=7 loops=1) + Index Cond: ((relname >= 'sql'::name) AND (relname < 'sqm'::name)) + Filter: ((relname ~~ 'sql%'::text) AND (relkind = 'r'::"char")) +--? -> Seq Scan on pg_namespace n (actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=10 loops=7) +--? Total runtime: [0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* ms +(8 rows) + +--set parallel parameter +set force_parallel_mode=on; +set parallel_setup_cost=0; +set parallel_tuple_cost=0.000005; +set max_parallel_workers_per_gather=2; +set min_parallel_table_scan_size=0; +set parallel_leader_participation=on; +-- nestloop +explain (costs off, analyse on) select schemaname, tablename from pg_tables where tablename like 'sql%' order by tablename; +--?.* +--?.* +--? Nested Loop Left Join (actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=7 loops=1) + Join Filter: (n.oid = c.relnamespace) +--? Rows Removed by Join Filter: [0-9]* +--? -> Index Scan using pg_class_relname_nsp_index on pg_class c (actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=7 loops=1) + Index Cond: ((relname >= 'sql'::name) AND (relname < 'sqm'::name)) + Filter: ((relname ~~ 'sql%'::text) AND (relkind = 'r'::"char")) +--? -> Gather (actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=10 loops=7) + Number of Workers: 1 +--? -> Parallel Seq Scan on pg_namespace n (actual time=[0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* rows=5 loops=14) +--? Total runtime: [0-9]*\.[0-9]*\.\.[0-9]*\.[0-9]* ms +(10 rows) + +--clean up +reset force_parallel_mode; +reset parallel_setup_cost; +reset parallel_tuple_cost; +reset max_parallel_workers_per_gather; +reset min_parallel_table_scan_size; +reset parallel_leader_participation; +reset enable_hashjoin; +reset enable_mergejoin; \ No newline at end of file diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out index ce9527eae..b18c12a44 100644 --- a/src/test/regress/expected/rangefuncs.out +++ b/src/test/regress/expected/rangefuncs.out @@ -61,6 +61,8 @@ SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%' ORDER BY name; enable_page_lsn_check | on enable_parallel_append | on enable_parallel_ddl | on + enable_parallel_hash | on + enable_partition_opfusion | off enable_partitionwise | off enable_pbe_optimization | on enable_prevent_job_task_startup | off @@ -85,7 +87,7 @@ SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%' ORDER BY name; enable_vector_engine | on enable_wdr_snapshot | off enable_xlog_prune | on -(81 rows) +(83 rows) CREATE TABLE foo2(fooid int, f2 int); INSERT INTO foo2 VALUES(1, 11); diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule index c25e49fb8..1eeca6d1a 100644 --- a/src/test/regress/parallel_schedule +++ b/src/test/regress/parallel_schedule @@ -29,11 +29,13 @@ test: misc test: stats test: alter_system_set +# parallel query +test: parallel_query parallel_nested_loop parallel_hashjoin parallel_append parallel_create_index + #dispatch from 13 test: function test: aggregates_part1 aggregates_part2 aggregates_part3 count_distinct_part1 count_distinct_part2 count_distinct_part3 count_distinct_part4 - test: hw_dfx_thread_status test: stable_function_shippable @@ -595,9 +597,6 @@ test: create_procedure create_function pg_compatibility postgres_fdw # autonomous transaction Test #test: autonomous_transaction -# parallel query -test: parallel_query parallel_nested_loop parallel_hashjoin parallel_append parallel_create_index - # gs_basebackup test: gs_basebackup diff --git a/src/test/regress/parallel_schedule3 b/src/test/regress/parallel_schedule3 index 78d29b941..ae1e81124 100644 --- a/src/test/regress/parallel_schedule3 +++ b/src/test/regress/parallel_schedule3 @@ -27,4 +27,4 @@ test: upsert_trigger_test upsert_explain test: upsert_clean # test parallel query -test: parallel_query parallel_append +test: parallel_query parallel_nested_loop parallel_hashjoin parallel_append parallel_create_index diff --git a/src/test/regress/sql/parallel_append.sql b/src/test/regress/sql/parallel_append.sql index e26057f8a..a14b13a29 100644 --- a/src/test/regress/sql/parallel_append.sql +++ b/src/test/regress/sql/parallel_append.sql @@ -31,14 +31,14 @@ set enable_parallel_append to on; ------------------------------------------- -- 1. union && union all ------------------------------------------- -explain select * from a union select * from b; -explain select * from a union all select * from b; -explain select * from a where a1 > 4 union select * from b where b1 < 6; -explain select * from a where a1 > 4 union all select * from b where b1 < 6; -explain select * from c where c1 in (select a1 from a union select b1 from b); -explain select * from (select * from a union all select * from b) as ta, c where ta.a1 = c.c1; -explain select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1; -explain select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from b) t, d where t.a1=d1 group by d.d1 order by 1,2; +explain (costs off) select * from a union select * from b; +explain (costs off) select * from a union all select * from b; +explain (costs off) select * from a where a1 > 4 union select * from b where b1 < 6; +explain (costs off) select * from a where a1 > 4 union all select * from b where b1 < 6; +explain (costs off) select * from c where c1 in (select a1 from a union select b1 from b); +explain (costs off) select * from (select * from a union all select * from b) as ta, c where ta.a1 = c.c1; +explain (costs off) select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1 order by 1,2,3,4,5,6; +explain (costs off) select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from b) t, d where t.a1=d1 group by d.d1 order by 1,2; select * from a union select * from b; select * from a union all select * from b; @@ -46,7 +46,7 @@ select * from a where a1 > 4 union select * from b where b1 < 6; select * from a where a1 > 4 union all select * from b where b1 < 6; select * from c where c1 in (select a1 from a union select b1 from b); select * from (select * from a union all select * from b) as ta, c where ta.a1 = c.c1; -select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1; +select * from d left outer join (select * from a union all select * from b) as t on d.d1=t.a1 order by 1,2,3,4,5,6; select d.d1, sum(d.d2), sum(t.a2) from (select * from a union all select * from b) t, d where t.a1=d1 group by d.d1 order by 1,2; -- set parallel_workers of table a to 0, subplan of seqscan on a should not be paralleled @@ -63,8 +63,8 @@ alter table a reset (parallel_workers); select * from c except select * from b where b1 >4; select * from c except all select * from b where b1 >4; -explain select * from c except select * from b where b1 >4; -explain select * from c except all select * from b where b1 >4; +explain (costs off) select * from c except select * from b where b1 >4; +explain (costs off) select * from c except all select * from b where b1 >4; --------------------------------------- @@ -73,8 +73,8 @@ explain select * from c except all select * from b where b1 >4; select * from e intersect select * from c; select * from e intersect all select * from c where c1 != 8; -explain select * from e intersect select * from c; -explain select * from e intersect all select * from c where c1 != 8; +explain (costs off) select * from e intersect select * from c; +explain (costs off) select * from e intersect all select * from c where c1 != 8; -------------------------------------- @@ -88,12 +88,12 @@ select * from b union all (select * from (select * from a union all select * fro select * from (select * from a union all select * from b)as x, (select * from d union all select* from e)as y where x.a1 = y.d1 order by 1, 2, 3, 4, 5, 6; -explain select * from e intersect (select * from a except select * from b union select * from c); -explain select d2 from d except all (select d2 from d except select c1 from c) union all select e1 from e; -explain select * from a union all (select * from b union select * from c where c1 < 5); -explain select * from a except select * from b union select * from c; -explain select * from b union all (select * from (select * from a union all select * from b)); -explain select * from (select * from a union all select * from b)as x, (select * from d union all select* from e)as y +explain (costs off) select * from e intersect (select * from a except select * from b union select * from c); +explain (costs off) select d2 from d except all (select d2 from d except select c1 from c) union all select e1 from e; +explain (costs off) select * from a union all (select * from b union select * from c where c1 < 5); +explain (costs off) select * from a except select * from b union select * from c; +explain (costs off) select * from b union all (select * from (select * from a union all select * from b)); +explain (costs off) select * from (select * from a union all select * from b)as x, (select * from d union all select* from e)as y where x.a1 = y.d1 order by 1, 2, 3, 4, 5, 6; ---------------------------------------- -- clean up diff --git a/src/test/regress/sql/parallel_hashjoin.sql b/src/test/regress/sql/parallel_hashjoin.sql index 3653b9309..10210218b 100644 --- a/src/test/regress/sql/parallel_hashjoin.sql +++ b/src/test/regress/sql/parallel_hashjoin.sql @@ -16,9 +16,11 @@ set enable_nestloop=off; explain (costs off) select * from parallel_hashjoin_test_a left outer join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id where parallel_hashjoin_test_a.id < 10 order by parallel_hashjoin_test_a.id; select * from parallel_hashjoin_test_a left outer join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id where parallel_hashjoin_test_a.id < 10 order by parallel_hashjoin_test_a.id; -- Forbid parallel Hash Right Join or Hash Full Join. -explain (costs off)select * from parallel_hashjoin_test_a right outer join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id order by parallel_hashjoin_test_a.id; -select * from parallel_hashjoin_test_a right outer join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id order by parallel_hashjoin_test_a.id; +explain (costs off)select * from parallel_hashjoin_test_a full join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id order by parallel_hashjoin_test_a.id limit 10; +select * from parallel_hashjoin_test_a full join parallel_hashjoin_test_b on parallel_hashjoin_test_a.id = parallel_hashjoin_test_b.id order by parallel_hashjoin_test_a.id limit 10; +drop table parallel_hashjoin_test_a; +drop table parallel_hashjoin_test_b; reset parallel_setup_cost; reset min_parallel_table_scan_size; reset parallel_tuple_cost; diff --git a/src/test/regress/sql/parallel_nested_loop.sql b/src/test/regress/sql/parallel_nested_loop.sql index 6029a9640..7cdd8f528 100644 --- a/src/test/regress/sql/parallel_nested_loop.sql +++ b/src/test/regress/sql/parallel_nested_loop.sql @@ -10,6 +10,7 @@ select * from parallel_nested_loop_test_a left outer join parallel_nested_loop_t set parallel_setup_cost = 0; set min_parallel_table_scan_size=0; set force_parallel_mode=on; +set parallel_tuple_cost= 0.1; explain (costs off) select * from parallel_nested_loop_test_a left outer join parallel_nested_loop_test_b on parallel_nested_loop_test_a.id = 1; select * from parallel_nested_loop_test_a left outer join parallel_nested_loop_test_b on parallel_nested_loop_test_a.id = 1; @@ -19,4 +20,5 @@ drop table parallel_nested_loop_test_b; reset parallel_setup_cost; reset min_parallel_table_scan_size; -reset force_parallel_mode; \ No newline at end of file +reset force_parallel_mode; +reset parallel_tuple_cost; \ No newline at end of file diff --git a/src/test/regress/sql/parallel_query.sql b/src/test/regress/sql/parallel_query.sql index e5ddd7402..7d284f5e9 100644 --- a/src/test/regress/sql/parallel_query.sql +++ b/src/test/regress/sql/parallel_query.sql @@ -115,4 +115,27 @@ reset parallel_tuple_cost; reset max_parallel_workers_per_gather; reset min_parallel_table_scan_size; reset parallel_leader_participation; -reset min_parallel_index_scan_size; \ No newline at end of file +reset min_parallel_index_scan_size; + +-- nestloop +set enable_hashjoin=off; +set enable_mergejoin=off; +explain (costs off, analyse on) select schemaname, tablename from pg_tables where tablename like 'sql%' order by tablename; +--set parallel parameter +set force_parallel_mode=on; +set parallel_setup_cost=0; +set parallel_tuple_cost=0.000005; +set max_parallel_workers_per_gather=2; +set min_parallel_table_scan_size=0; +set parallel_leader_participation=on; +-- nestloop +explain (costs off, analyse on) select schemaname, tablename from pg_tables where tablename like 'sql%' order by tablename; +--clean up +reset force_parallel_mode; +reset parallel_setup_cost; +reset parallel_tuple_cost; +reset max_parallel_workers_per_gather; +reset min_parallel_table_scan_size; +reset parallel_leader_participation; +reset enable_hashjoin; +reset enable_mergejoin; \ No newline at end of file