!431 parallel hash join: inner join

Merge pull request !431 from 吴岳川/master
This commit is contained in:
opengauss-bot 2020-11-24 17:32:25 +08:00 committed by Gitee
commit 49b91b56e1
74 changed files with 5056 additions and 820 deletions

View File

@ -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.|

View File

@ -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);

View File

@ -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);

View File

@ -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();
}

View File

@ -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",

View File

@ -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 -

View File

@ -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

View File

@ -0,0 +1,584 @@
/*-------------------------------------------------------------------------
*
* sharedtuplestore.cpp
* 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
* Portions Copyright (c) 2020 Huawei Technologies Co.,Ltd.
*
* IDENTIFICATION
* src/common/backend/utils/sort/sharedtuplestore.cpp
*
*-------------------------------------------------------------------------
*/
#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;
(void)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, const 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];
(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;
}
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, "", "");
}

View File

@ -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 ||

View File

@ -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<false>(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<false>(es, nbatch, nbatch_original, nbuckets, spacePeakKb);
}
}
}
}

View File

@ -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

View File

@ -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;

View File

@ -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 */);
}
}
}

View File

@ -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;

View File

@ -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))

View File

@ -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);

View File

@ -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);

View File

@ -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);
}

View File

@ -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);

View File

@ -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;
}
/*

View File

@ -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
*

View File

@ -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);

View File

@ -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);

View File

@ -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);
}
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -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;

View File

@ -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
*

View File

@ -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
*

View File

@ -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;
}

View File

@ -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);
}
}
}

View File

@ -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 <bool isonlyindex> 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<false>((Node *)scan, ftype);
}
} else {
IndexOnlyScan* scan = (IndexOnlyScan *)top_plan;
if (scan->scan.isPartTbl == true) {
ftype = checkFusionPartitionNumber<true>((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<true, false>((Node *)linitial(node->plans), params);
FusionType ttype = checkFusionIndexScan<true, false>((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<false>((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<true, false>((Node *)linitial(node->plans), params);
FusionType ttype = checkFusionIndexScan<true, false>((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<false>((Node *)indexscan, ftype);
}
return ftype;
}

View File

@ -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(&parallel_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(&parallel_scan->phs_nallocated, 0);
}
/* ----------------
* heap_beginscan_parallel - join a parallel scan
*

View File

@ -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) {

View File

@ -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;

View File

@ -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

View File

@ -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));
}

View File

@ -156,7 +156,9 @@ static const char *BuiltinTrancheNames[] = {
/* LWTRANCHE_GTT_CTL */
"GlobalTempTableControl",
"PLdebugger",
"parallel_append"
"SharedTupleStore",
"parallel_append",
"ParallelHashJoinLock"
};
static void RegisterLWLockTranches(void);

View File

@ -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);

View File

@ -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 */

View File

@ -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 */

View File

@ -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 */

View File

@ -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 */

View File

@ -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);
/*

View File

@ -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 */

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;
/* ----------------

View File

@ -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;
/* ----------------

View File

@ -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;

View File

@ -68,6 +68,10 @@ public:
Relation m_rel;
Relation m_parentRel;
Partition m_partRel;
TupleTableSlot* m_reslot;
TupleDesc m_tupDesc;

View File

@ -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 {

View File

@ -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 */

View File

@ -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);

View File

@ -143,8 +143,9 @@ enum BuiltinTrancheIds {
LWTRANCHE_EXTEND, // For general 3rd plugin
LWTRANCHE_GTT_CTL, // For GTT
LWTRANCHE_PLDEBUG, // For Pldebugger
LWTRANCHE_SHARED_TUPLESTORE,
LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_PARALLEL_HASH_JOIN,
/*
* Each trancheId above should have a corresponding item in BuiltinTrancheNames;
*/

View File

@ -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 */

82
src/include/utils/dsa.h Normal file
View File

@ -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 */

View File

@ -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);

View File

@ -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_ */

View File

@ -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, const void* meta_data, MinimalTuple tuple);
extern MinimalTuple sts_parallel_scan_next(SharedTuplestoreAccessor* accessor, void* meta_data);
#endif /* SHAREDTUPLESTORE_H */

View File

@ -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=.*)

View File

@ -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

View File

@ -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;

View File

@ -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;

View File

@ -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;
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;

View File

@ -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);

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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;

View File

@ -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;
reset force_parallel_mode;
reset parallel_tuple_cost;

View File

@ -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;
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;

View File

@ -1558,6 +1558,8 @@ SharedInvalRelcacheMsg
SharedInvalRelmapMsg
SharedInvalSmgrMsg
SharedInvalidationMessage
SharedTuplestore
SharedTuplestoreAccessor
ShellTypeInfo
ShmemIndexEnt
ShutdownInformation