WIP: [[Parallel] Shared] Hash
Hi hackers,
In PostgreSQL 9.6, hash joins can be parallelised under certain
conditions, but a copy of the hash table is built in every
participating backend. That means that memory and CPU time are
wasted. In many cases, that's OK: if the hash table contents are
small and cheap to compute, then we don't really care, we're just
happy that the probing can be done in parallel. But in cases where
the hash table is large and/or expensive to build, we could do much
better. I am working on that problem.
To recap the situation in 9.6, a hash join can appear below a Gather
node and it looks much the same as a non-parallel hash join except
that it has a partial outer plan:
-> Hash Join
-> <partial outer plan>
-> Hash
-> <non-partial parallel-safe inner plan>
A partial plan is one that has some kind of 'scatter' operation as its
ultimate source of tuples. Currently the only kind of scatter
operation is a Parallel Seq Scan (but see also the Parallel Index Scan
and Parallel Bitmap Scan proposals). The scatter operation enables
parallelism in all the executor nodes above it, as far as the
enclosing 'gather' operation which must appear somewhere above it.
Currently the only kind of gather operation is a Gather node (but see
also the Gather Merge proposal which adds a new one).
The inner plan is built from a non-partial parallel-safe path and will
be run in every worker.
Note that a Hash Join node in 9.6 isn't parallel-aware itself: it's
not doing anything special at execution time to support parallelism.
The planner has determined that correct partial results will be
produced by this plan, but the executor nodes are blissfully unaware
of parallelism.
PROPOSED NEW PLAN VARIANTS
Shortly I will post a patch which introduces two new hash join plan
variants that are parallel-aware:
1. Parallel Hash Join with Shared Hash
-> Parallel Hash Join
-> <partial outer plan>
-> Shared Hash
-> <non-partial parallel-safe inner plan>
In this case, there is only one copy of the hash table and only one
participant loads it. The other participants wait patiently for one
chosen backend to finish building the hash table, and then they all
wake up and probe.
Call the number of participants P, being the number of workers + 1
(for the leader). Compared to a non-shared hash plan, we avoid
wasting CPU and IO resources running P copies of the inner plan in
parallel (something that is not well captured in our costing model for
parallel query today), and we can allow ourselves to use a hash table
P times larger while sticking to the same overall space target of
work_mem * P.
2. Parallel Hash Join with Parallel Shared Hash
-> Parallel Hash Join
-> <partial outer plan>
-> Parallel Shared Hash
-> <partial inner plan>
In this case, the inner plan is run in parallel by all participants.
We have the advantages of a shared hash table as described above, and
now we can also divide the work of running the inner plan and hashing
the resulting tuples by P participants. Note that Parallel Shared
Hash is acting as a special kind of gather operation that is the
counterpart to the scatter operation contained in the inner plan.
PERFORMANCE
So far I have been unable to measure any performance degradation
compared with unpatched master for hash joins with non-shared hash.
That's good because it means that I didn't slow existing plans down
when I introduced a bunch of conditional branches to existing hash
join code.
Laptop testing shows greater than 2x speedups on several of the TPC-H
queries with single batches, and no slowdowns. I will post test
numbers on big rig hardware in the coming weeks when I have the
batching code in more complete and stable shape.
IMPLEMENTATION
I have taken the approach of extending the existing hash join
algorithm, rather than introducing separate hash join executor nodes
or a fundamentally different algorithm. Here's a short description of
what the patch does:
1. SHARED HASH TABLE
To share data between participants, the patch uses two other patches I
have proposed: DSA areas[1]/messages/by-id/CAEepm=1z5WLuNoJ80PaCvz6EtG9dN0j-KuHcHtU6QEfcPP5-qA@mail.gmail.com, which provide a higher level interface
to DSM segments to make programming with processes a little more like
programming with threads, and in particular a per-parallel-query DSA
area[2]/messages/by-id/CAEepm=0HmRefi1+xDJ99Gj5APHr8Qr05KZtAxrMj8b+ay3o6sA@mail.gmail.com that is made available for any executor node that needs some
shared work space.
The patch uses atomic operations to push tuples into the hash table
buckets while building, rehashing and loading, and then the hash table
is immutable during probing (except for match flags used to implement
outer joins). The existing memory chunk design is retained for dense
allocation of tuples, which provides a convenient way to rehash the
table when its size changes.
2. WORK COORDINATION
To coordinate parallel work, this patch uses two other patches:
barriers[3]/messages/by-id/CAEepm=2_y7oi01OjA_wLvYcWMc9_d=LaoxrY3eiROCZkB_qakA@mail.gmail.com, to implement a 'barrier' or 'phaser' synchronisation
primitive, and those in turn use the condition variables proposed by
Robert Haas.
Barriers provide a way for participants to break work up into phases
that they unanimously agree to enter together, which is a basic
requirement for parallelising hash joins. It is not safe to insert
into the hash table until exactly one participant has created it; it
is not safe to probe the hash table until all participants have
finished inserting into it; it is not safe to scan it for unmatched
tuples until all participants have finished probing it; it is not safe
to discard it and start loading the next batch until ... you get the
idea. You could also construct appropriate synchronisation using
various other interlocking primitives or flow control systems, but
fundamentally these wait points would exist at some level, and I think
this way is quite clean and simple. YMMV.
If we had exactly W workers and the leader didn't participate, then we
could use a simple simple pthread- or MPI-style barrier without an
explicit notion of 'phase'. We would simply take the existing hash
join code, add the shared hash table, add barrier waits at various
points and make sure that all participants always hit all of those
points in the same order, and it should All Just Work. But we have a
variable party size and a dual-role leader process, and I want to
highlight the specific problems that causes here because they increase
the patch size significantly:
Problem 1: We don't know how many workers will actually start. We
know how many were planned, but at execution time we may have
exhausted limits and actually get a smaller number. So we can't use
"static" barriers like the classic barriers in POSIX or MPI where the
group size is known up front. We need "dynamic" barriers with attach
and detach operations. As soon as you have varying party size you
need some kind of explicit model of the current phase, so that a new
participant can know what to do when it joins. For that reason, this
patch uses a phase number to track progress through the parallel hash
join. See MultiExecHash and ExecHashJoin which have switch statements
allowing a newly joined participant to synchronise their own state
machine and program counter with the phase.
Problem 2: One participant is not like the others: Gather may or may
not decide to run its subplan directly if the worker processes aren't
producing any tuples (and the proposed Gather Merge is the same). The
problem is that it also needs to consume tuples from the fixed-size
queues of the regular workers. A deadlock could arise if the leader's
plan blocks waiting for other participants while another participant
has filled its output queue and is waiting for the leader to consume.
One way to avoid such deadlocks is to follow the rule that the leader
should never wait for other participants if there is any possibility
that they have emitted tuples. The simplest way to do that would be
to have shared hash plans refuse to run in the leader by returning
NULL to signal the end of this partial tuple stream, but then we'd
lose a CPU compared to non-shared hash plans. The latest point the
leader can exit while respecting that rule is at the end of probing
the first batch. That is the approach taken by the patch currently.
See ExecHashCheckForEarlyExit for logic and discussion. It would be
better to be able to use the leader in later batches too, but as far
as I can see that'd require changes that are out of scope for this
patch. One idea would be an executor protocol change allowing plans
running in the leader to detach and yield, saying 'I have no further
tuples right now, but I'm not finished; try again later', and then
reattach when you call it back. Clearly that sails close to
asynchronous execution territory.
Problem 3: If the leader drops out after the first batch to solve
problem 2, then it may leave behind batch files which must be
processed by other participants. I had originally planned to defer
work on batch file sharing until a later iteration, thinking that it
would be a nice performance improvement to redistribute work from
uneven batch files, but it turns out to be necessary for correct
results because of participants exiting early. I am working on a very
simple batch sharing system to start with... Participants still
generate their own batch files, and then new operations BufFileExport
and BufFileImport are used to grant read-only access to the BufFile to
other participants. Each participant reads its own batch files
entirely and then tries to read from every other participant's batch
files until they are all exhausted, using a shared read head. The
per-tuple locking granularity, extra seeking and needless buffering in
every backend on batch file reads aren't great, and I'm still figuring
out temporary file cleanup/ownership semantics. There may be an
opportunity to make use of 'unified' BufFile concepts from Peter
Geoghegan's work, or create some new reusable shared tuple spilling
infrastructure.
3. COSTING
For now, I have introduced a GUC called cpu_shared_tuple_cost which
provides a straw-man model of the overhead of exchanging tuples via a
shared hash table, and the extra process coordination required. If
it's zero then a non-shared hash plan (ie multiple copies) has the
same cost as a shared hash plan, even though the non-shared hash plan
wastefully runs P copies of the plan. If cost represents runtime and
and we assume perfectly spherical cows running without interference
from each other, that makes some kind of sense, but it doesn't account
for the wasted resources and contention caused by running the same
plan in parallel. I don't know what to do about that yet. If
cpu_shared_tuple_cost is a positive number, as it probably should be
(more on that later), then shared hash tables look more expensive than
non-shared ones, which is technically true (CPU cache sharing etc) but
unhelpful because what you lose there you tend to gain by not running
all those plans in parallel. In other words cpu_shared_tuple_cost
doesn't really model the cost situation at all well, but it's a useful
GUC for development purposes for now as positive and negative numbers
can be used to turn the feature on and off for testing... As for
work_mem, it seems to me that 9.6 already established that work_mem is
a per participant limit, and it would be only fair to let a shared
plan use a total of work_mem * P too. I am still working on work_mem
accounting and reporting. Accounting for the parallelism in parallel
shared hash plans is easy though: their estimated tuple count is
already divided by P in the underlying partial path, and that is a
fairly accurate characterisation of what's going to happen at
execution time: it's often going to go a lot faster, and those plans
are the real goal of this work.
STATUS
Obviously this is a work in progress. I am actively working on the following:
* rescan
* batch number increases
* skew buckets
* costing model and policy/accounting for work_mem
* shared batch file reading
* preloading next batch
* debugging and testing
* tidying and refactoring
The basic approach is visible and simple cases are working though, so
I am submitting this WIP work for a round of review in the current
commitfest and hoping to get some feedback and ideas. I will post the
patch in a follow-up email shortly... Thanks for reading!
[1]: /messages/by-id/CAEepm=1z5WLuNoJ80PaCvz6EtG9dN0j-KuHcHtU6QEfcPP5-qA@mail.gmail.com
[2]: /messages/by-id/CAEepm=0HmRefi1+xDJ99Gj5APHr8Qr05KZtAxrMj8b+ay3o6sA@mail.gmail.com
[3]: /messages/by-id/CAEepm=2_y7oi01OjA_wLvYcWMc9_d=LaoxrY3eiROCZkB_qakA@mail.gmail.com
--
Thomas Munro
http://www.enterprisedb.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Thomas Munro <thomas.munro@enterprisedb.com> wrote:
The basic approach is visible and simple cases are working though, so
I am submitting this WIP work for a round of review in the current
commitfest and hoping to get some feedback and ideas. I will post the
patch in a follow-up email shortly...
Aloha,
Please find a WIP patch attached. Everything related to batch reading
is not currently in a working state, which breaks multi-batch joins,
but many single batch cases work correctly. In an earlier version I
had multi-batch joins working but was before I started tackling
problems 2 and 3 listed in my earlier message. There is some error
handling and resource cleanup missing, and doubtless some cases not
handled correctly. But I thought it would be good to share this
development snapshot for discussion, so I'm posting this as is, and
will post an updated version when I've straightened out the batching
code some more.
To apply parallel-hash-v1, first apply the following patches, in this order:
condition-variable-v3.patch [1]/messages/by-id/CA+Tgmoaj2aPti0yho7FeEf2qt-JgQPRWb0gci_o1Hfr=C56Xng@mail.gmail.com
remove-useless-barrier-header-v2.patch [2]/messages/by-id/CAEepm=1wrrzxh=SRCF_Hk4SZQ9BULy1vWsicx0EbgUf0B85vZQ@mail.gmail.com
barrier-v3.patch [2]/messages/by-id/CAEepm=1wrrzxh=SRCF_Hk4SZQ9BULy1vWsicx0EbgUf0B85vZQ@mail.gmail.com
dsa-v4.patch [3]/messages/by-id/CAEepm=1z5WLuNoJ80PaCvz6EtG9dN0j-KuHcHtU6QEfcPP5-qA@mail.gmail.com
dsa-area-for-executor-v1.patch [4]/messages/by-id/CAEepm=0HmRefi1+xDJ99Gj5APHr8Qr05KZtAxrMj8b+ay3o6sA@mail.gmail.com
When applying dsa-v4 on top of barrier-v3, it will reject a hunk in
src/backend/storage/ipc/Makefile where they both add their object
file. Simply add dsa.o to OBJS manually.
Then you can apply parallel-hash-v1.patch, which is attached to this message.
[1]: /messages/by-id/CA+Tgmoaj2aPti0yho7FeEf2qt-JgQPRWb0gci_o1Hfr=C56Xng@mail.gmail.com
[2]: /messages/by-id/CAEepm=1wrrzxh=SRCF_Hk4SZQ9BULy1vWsicx0EbgUf0B85vZQ@mail.gmail.com
[3]: /messages/by-id/CAEepm=1z5WLuNoJ80PaCvz6EtG9dN0j-KuHcHtU6QEfcPP5-qA@mail.gmail.com
[4]: /messages/by-id/CAEepm=0HmRefi1+xDJ99Gj5APHr8Qr05KZtAxrMj8b+ay3o6sA@mail.gmail.com
--
Thomas Munro
http://www.enterprisedb.com
Attachments:
parallel-hash-v1.patchapplication/octet-stream; name=parallel-hash-v1.patchDownload
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 0a669d9..1e7d369 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -1023,7 +1023,10 @@ ExplainNode(PlanState *planstate, List *ancestors,
pname = sname = "Limit";
break;
case T_Hash:
- pname = sname = "Hash";
+ if (((Hash *) plan)->shared_table)
+ pname = sname = "Shared Hash";
+ else
+ pname = sname = "Hash";
break;
default:
pname = sname = "???";
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 72bacd5..2d1ff2a 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -27,6 +27,7 @@
#include "executor/executor.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
+#include "executor/nodeHashJoin.h"
#include "executor/nodeSeqscan.h"
#include "executor/tqueue.h"
#include "nodes/nodeFuncs.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
break;
+ case T_HashJoinState:
+ ExecHashJoinEstimate((HashJoinState *) planstate,
+ e->pcxt);
+ break;
default:
break;
}
@@ -255,6 +260,9 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
break;
+ case T_HashJoinState:
+ ExecHashJoinInitializeDSM((HashJoinState *) planstate,
+ d->pcxt);
default:
break;
}
@@ -724,6 +732,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
break;
+ case T_HashJoinState:
+ ExecHashJoinInitializeWorker((HashJoinState *) planstate,
+ toc);
+ break;
default:
break;
}
diff --git a/src/backend/executor/nodeHash.c b/src/backend/executor/nodeHash.c
index 6375d9b..1cc7f59 100644
--- a/src/backend/executor/nodeHash.c
+++ b/src/backend/executor/nodeHash.c
@@ -25,6 +25,7 @@
#include <limits.h>
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "catalog/pg_statistic.h"
#include "commands/tablespace.h"
#include "executor/execdebug.h"
@@ -32,12 +33,13 @@
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
#include "miscadmin.h"
+#include "pgstat.h"
+#include "port/atomics.h"
#include "utils/dynahash.h"
#include "utils/memutils.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
-
static void ExecHashIncreaseNumBatches(HashJoinTable hashtable);
static void ExecHashIncreaseNumBuckets(HashJoinTable hashtable);
static void ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node,
@@ -47,8 +49,30 @@ static void ExecHashSkewTableInsert(HashJoinTable hashtable,
uint32 hashvalue,
int bucketNumber);
static void ExecHashRemoveNextSkewBucket(HashJoinTable hashtable);
+static void ExecHashRebucket(HashJoinTable hashtable);
+static void ExecHashTableComputeOptimalBuckets(HashJoinTable hashtable);
+
+static void add_tuple_count(HashJoinTable hashtable, int count,
+ bool secondary);
+static HashJoinTuple next_tuple_in_bucket(HashJoinTable table,
+ HashJoinTuple tuple);
+static HashJoinTuple first_tuple_in_skew_bucket(HashJoinTable table,
+ int skew_bucket_no);
+static HashJoinTuple first_tuple_in_skew_bucket(HashJoinTable table,
+ int bucket_no);
+static void insert_tuple_into_bucket(HashJoinTable table, int bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_pointer);
+static void insert_tuple_into_skew_bucket(HashJoinTable table,
+ int bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_pointer);
static void *dense_alloc(HashJoinTable hashtable, Size size);
+static void *dense_alloc_shared(HashJoinTable hashtable, Size size,
+ dsa_pointer *chunk_shared,
+ bool secondary);
+
/* ----------------------------------------------------------------
* ExecHash
@@ -64,6 +88,100 @@ ExecHash(HashState *node)
}
/* ----------------------------------------------------------------
+ * ExecHashCheckForEarlyExit
+ *
+ * return true if this process needs to abandon work on the
+ * hash join to avoid a deadlock
+ * ----------------------------------------------------------------
+ */
+bool
+ExecHashCheckForEarlyExit(HashJoinTable hashtable)
+{
+ /*
+ * The golden rule of leader deadlock avoidance: since leader processes
+ * have two separate roles, namely reading from worker queues AND executing
+ * the same plan as workers, we must never allow a leader to wait for
+ * workers if there is any possibility those workers have emitted tuples.
+ * Otherwise we could get into a situation where a worker fills up its
+ * output tuple queue and begins waiting for the leader to read, while
+ * the leader is busy waiting for the worker.
+ *
+ * Parallel hash joins with shared tables are inherently susceptible to
+ * such deadlocks because there are points at which all participants must
+ * wait (you can't start check for unmatched tuples in the hash table until
+ * probing has completed in all workers, etc).
+ *
+ * So we follow these rules:
+ *
+ * 1. If there are workers participating, the leader MUST NOT not
+ * participate in any further work after probing the first batch, so
+ * that it never has to wait for workers that might have emitted
+ * tuples.
+ *
+ * 2. If there are no workers participating, the leader MUST run all the
+ * batches to completion, because that's the only way for the join
+ * to complete. There is no deadlock risk if there are no workers.
+ *
+ * 3. Workers MUST NOT participate if the hashing phase has finished by
+ * the time they have joined, so that the leader can reliably determine
+ * whether there are any workers running when it comes to the point
+ * where it must choose between 1 and 2.
+ *
+ * In other words, if the leader makes it all the way through hashing and
+ * probing before any workers show up, then the leader will run the whole
+ * hash join on its own. If workers do show up any time before hashing is
+ * finished, the leader will stop executing the join after helping probe
+ * the first batch. In the unlikely event of the first worker showing up
+ * after the leader has finished hashing, it will exit because it's too
+ * late, the leader has already decided to do all the work alone.
+ */
+
+ if (!IsParallelWorker())
+ {
+ /* Running in the leader process. */
+ if (BarrierPhase(&hashtable->shared->barrier) == PHJ_PHASE_PROBING &&
+ hashtable->shared->at_least_one_worker)
+ {
+ /* Abandon ship due to rule 1. There are workers running. */
+ hashtable->detached_early = true;
+ }
+ else
+ {
+ /*
+ * Continue processing due to rule 2. There are no workers, and
+ * any workers that show up later will abandon ship.
+ */
+ }
+ }
+ else
+ {
+ /* Running in a worker process. */
+ if (hashtable->attached_at_phase < PHJ_PHASE_PROBING)
+ {
+ /*
+ * Advertise that there are workers, so that the leader can
+ * choose between rules 1 and 2. It's OK that several workers can
+ * write to this variable without immediately memory
+ * synchronization, because the leader will only read it in a later
+ * phase (see above).
+ */
+ hashtable->shared->at_least_one_worker = true;
+ }
+ else
+ {
+ /* Abandon ship due to rule 3. */
+ hashtable->detached_early = true;
+ }
+ }
+
+ /* If we decided to exit early, detach now. */
+ if (hashtable->detached_early)
+ BarrierDetach(&hashtable->shared->barrier);
+
+ return hashtable->detached_early;
+}
+
+/* ----------------------------------------------------------------
* MultiExecHash
*
* build hash table for hashjoin, doing partitioning if more
@@ -79,6 +197,7 @@ MultiExecHash(HashState *node)
TupleTableSlot *slot;
ExprContext *econtext;
uint32 hashvalue;
+ Barrier *barrier = NULL;
/* must provide our own instrumentation support */
if (node->ps.instrument)
@@ -90,6 +209,55 @@ MultiExecHash(HashState *node)
outerNode = outerPlanState(node);
hashtable = node->hashtable;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Synchronize parallel hash table builds. At this stage we know that
+ * the shared hash table has been created, but we don't know if our
+ * peers are still in MultiExecHash and if so how far through. We use
+ * the phase to synchronize with them.
+ */
+ barrier = &hashtable->shared->barrier;
+
+ switch (BarrierPhase(barrier))
+ {
+ case PHJ_PHASE_INIT:
+ /* ExecHashTableCreate already handled this phase. */
+ Assert(false);
+ case PHJ_PHASE_CREATING:
+ /* Wait for serial phase, and then either hash or wait. */
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_CREATING))
+ goto hash;
+ else if (node->ps.plan->parallel_aware)
+ goto hash;
+ else
+ goto post_hash;
+ case PHJ_PHASE_HASHING:
+ /* Hashing is already underway. Can we join in? */
+ if (node->ps.plan->parallel_aware)
+ goto hash;
+ else
+ goto post_hash;
+ case PHJ_PHASE_RESIZING:
+ /* Can't help with serial phase. */
+ goto post_resize;
+ case PHJ_PHASE_REBUCKETING:
+ /* Rebucketing is in progress. Let's help do that. */
+ goto rebucket;
+ default:
+ /* The hash table building work is already finished. */
+ goto finish;
+ }
+ }
+
+ hash:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Make sure our local hashtable is up-to-date so we can hash. */
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_HASHING);
+ ExecHashUpdate(hashtable);
+ }
+
/*
* set expression context
*/
@@ -123,22 +291,98 @@ MultiExecHash(HashState *node)
else
{
/* Not subject to skew optimization, so insert normally */
- ExecHashTableInsert(hashtable, slot, hashvalue);
+ ExecHashTableInsert(hashtable, slot, hashvalue, false);
}
- hashtable->totalTuples += 1;
+ /*
+ * Shared tuple counters are managed by dense_alloc_shared. For
+ * private hash tables we maintain the counter here.
+ */
+ if (!HashJoinTableIsShared(hashtable))
+ hashtable->totalTuples += 1;
}
}
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Update shared tuple count for the current chunk. Other chunks are
+ * accounted for already, when new chunks are allocated.
+ */
+ if (hashtable->primary_chunk != NULL)
+ add_tuple_count(hashtable, hashtable->primary_chunk->ntuples,
+ false);
+ }
+
+ post_hash:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ bool elected_to_resize;
+
+ /*
+ * Wait for all backends to finish hashing. If only one worker is
+ * running the hashing phase because of a non-partial inner plan, the
+ * other workers will pile up here waiting. If multiple worker are
+ * hashing, they should finish close to each other in time.
+ */
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_HASHING);
+ elected_to_resize = BarrierWait(barrier, WAIT_EVENT_HASH_HASHING);
+ /*
+ * Resizing is a serial phase. All but one should skip ahead to
+ * rebucketing, but all workers should update their copy of the shared
+ * tuple count with the final total first.
+ */
+ hashtable->totalTuples =
+ pg_atomic_read_u64(&hashtable->shared->total_primary_tuples);
+ if (!elected_to_resize)
+ goto post_resize;
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_RESIZING);
+ }
+
/* resize the hash table if needed (NTUP_PER_BUCKET exceeded) */
- if (hashtable->nbuckets != hashtable->nbuckets_optimal)
- ExecHashIncreaseNumBuckets(hashtable);
+ ExecHashIncreaseNumBuckets(hashtable);
+
+ post_resize:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_RESIZING);
+ BarrierWait(&hashtable->shared->barrier,
+ WAIT_EVENT_HASH_RESIZING);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_REBUCKETING);
+ }
+
+ rebucket:
+ /* If the table was resized, insert tuples into the new buckets. */
+ ExecHashUpdate(hashtable);
+ ExecHashRebucket(hashtable);
/* Account for the buckets in spaceUsed (reported in EXPLAIN ANALYZE) */
- hashtable->spaceUsed += hashtable->nbuckets * sizeof(HashJoinTuple);
+ hashtable->spaceUsed += hashtable->nbuckets * sizeof(HashJoinBucketHead);
if (hashtable->spaceUsed > hashtable->spacePeak)
hashtable->spacePeak = hashtable->spaceUsed;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_REBUCKETING);
+ BarrierWait(barrier, WAIT_EVENT_HASH_REBUCKETING);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_PROBING);
+ }
+
+ finish:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * All hashing work has finished. The other workers may be probing or
+ * processing unmatched tuples for the initial batch, or dealing with
+ * later batches. The next synchronization point is in ExecHashJoin's
+ * HJ_BUILD_HASHTABLE case, which will figure that out and synchronize
+ * its local state machine with the parallel processing group's phase.
+ */
+ Assert(BarrierPhase(barrier) >= PHJ_PHASE_PROBING);
+ ExecHashUpdate(hashtable);
+ }
+
/* must provide our own instrumentation support */
+ /* TODO: report only the tuples that WE hashed here? */
if (node->ps.instrument)
InstrStopNode(node->ps.instrument, hashtable->totalTuples);
@@ -243,8 +487,9 @@ ExecEndHash(HashState *node)
* ----------------------------------------------------------------
*/
HashJoinTable
-ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
+ExecHashTableCreate(HashState *state, List *hashOperators, bool keepNulls)
{
+ Hash *node;
HashJoinTable hashtable;
Plan *outerNode;
int nbuckets;
@@ -261,6 +506,7 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
* "outer" subtree of this node, but the inner relation of the hashjoin).
* Compute the appropriate size of the hash table.
*/
+ node = (Hash *) state->ps.plan;
outerNode = outerPlan(node);
ExecChooseHashTableSize(outerNode->plan_rows, outerNode->plan_width,
@@ -305,7 +551,13 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
hashtable->spaceUsedSkew = 0;
hashtable->spaceAllowedSkew =
hashtable->spaceAllowed * SKEW_WORK_MEM_PERCENT / 100;
- hashtable->chunks = NULL;
+ hashtable->primary_chunk = NULL;
+ hashtable->secondary_chunk = NULL;
+ hashtable->chunks_to_rebucket = NULL;
+ hashtable->primary_chunk_shared = InvalidDsaPointer;
+ hashtable->secondary_chunk_shared = InvalidDsaPointer;
+ hashtable->area = state->ps.state->es_query_area;
+ hashtable->shared = state->shared_table_data;
#ifdef HJDEBUG
printf("Hashjoin %p: initial nbatch = %d, nbuckets = %d\n",
@@ -368,23 +620,101 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
PrepareTempTablespaces();
}
- /*
- * Prepare context for the first-scan space allocations; allocate the
- * hashbucket array therein, and set each bucket "empty".
- */
- MemoryContextSwitchTo(hashtable->batchCxt);
+ MemoryContextSwitchTo(oldcxt);
- hashtable->buckets = (HashJoinTuple *)
- palloc0(nbuckets * sizeof(HashJoinTuple));
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier;
- /*
- * Set up for skew optimization, if possible and there's a need for more
- * than one batch. (In a one-batch join, there's no point in it.)
- */
- if (nbatch > 1)
- ExecHashBuildSkewHash(hashtable, node, num_skew_mcvs);
+ /*
+ * Attach to the barrier. The corresponding detach operation is in
+ * ExecHashTableDestroy.
+ */
+ barrier = &hashtable->shared->barrier;
+ hashtable->attached_at_phase = BarrierAttach(barrier);
- MemoryContextSwitchTo(oldcxt);
+ /*
+ * So far we have no idea whether there are any other workers, and if
+ * so, what phase they are working on. The only thing we care about
+ * at this point is whether someone has already created the shared
+ * hash table yet. If not, one backend will be elected to do that
+ * now.
+ */
+ if (BarrierPhase(barrier) == PHJ_PHASE_INIT)
+ {
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_INIT))
+ {
+ /* Serial phase: create the hash tables */
+ Size bytes;
+ HashJoinBucketHead *buckets;
+ int i;
+ SharedHashJoinTable shared;
+ dsa_area *area;
+
+ shared = hashtable->shared;
+ area = hashtable->area;
+ bytes = nbuckets * sizeof(HashJoinBucketHead);
+
+ /* Allocate the primary and secondary hash tables. */
+ shared->primary_buckets = dsa_allocate(area, bytes);
+ shared->secondary_buckets = dsa_allocate(area, bytes);
+ if (!DsaPointerIsValid(shared->primary_buckets) ||
+ !DsaPointerIsValid(shared->secondary_buckets))
+ ereport(ERROR,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("out of memory")));
+
+ /* Set up primary table's buckets. */
+ buckets = dsa_get_address(area, shared->primary_buckets);
+ for (i = 0; i < nbuckets; ++i)
+ dsa_pointer_atomic_init(&buckets[i].shared,
+ InvalidDsaPointer);
+ /* Set up secondary table's buckets. */
+ buckets = dsa_get_address(area, shared->secondary_buckets);
+ for (i = 0; i < nbuckets; ++i)
+ dsa_pointer_atomic_init(&buckets[i].shared,
+ InvalidDsaPointer);
+
+ /* Initialize the rest of parallel_state. */
+ hashtable->shared->nbuckets = nbuckets;
+ pg_atomic_write_u32(&hashtable->shared->next_unmatched_bucket,
+ 0);
+ /* TODO: ExecHashBuildSkewHash */
+
+ ExecHashJoinResetBatchReaders(hashtable);
+
+ /*
+ * The backend-local pointers in hashtable will be set up by
+ * ExecHashUpdate, at each point where they might have
+ * changed.
+ */
+ }
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_CREATING);
+ /* The next synchronization point is in MultiExecHash. */
+ }
+ }
+ else
+ {
+ /*
+ * Prepare context for the first-scan space allocations; allocate the
+ * hashbucket array therein, and set each bucket "empty".
+ */
+ MemoryContextSwitchTo(hashtable->batchCxt);
+
+ hashtable->buckets = (HashJoinBucketHead *)
+ palloc0(nbuckets * sizeof(HashJoinBucketHead));
+
+ MemoryContextSwitchTo(oldcxt);
+
+ /*
+ * Set up for skew optimization, if possible and there's a need for
+ * more than one batch. (In a one-batch join, there's no point in
+ * it.)
+ */
+ if (nbatch > 1)
+ ExecHashBuildSkewHash(hashtable, node, num_skew_mcvs);
+ }
return hashtable;
}
@@ -564,6 +894,49 @@ ExecHashTableDestroy(HashJoinTable hashtable)
{
int i;
+ /* Detached, if we haven't already. */
+ if (HashJoinTableIsShared(hashtable) && !hashtable->detached_early)
+ {
+ Barrier *barrier = &hashtable->shared->barrier;
+
+ /*
+ * We can't make any assertions about the phase here, because we could
+ * be destroyed mid-probing due to a Limit clause, or after running
+ * out of work, or as a leading having decided to exit early. Instead
+ * we just detach from the barrier, and let the last participant to
+ * detach clean up.
+ */
+
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_DESTROY))
+ {
+ /* Serial: free the tables */
+ if (DsaPointerIsValid(hashtable->shared->primary_buckets))
+ {
+ dsa_free(hashtable->area,
+ hashtable->shared->primary_buckets);
+ hashtable->shared->primary_buckets = InvalidDsaPointer;
+ }
+ if (DsaPointerIsValid(hashtable->shared->secondary_buckets))
+ {
+ dsa_free(hashtable->area,
+ hashtable->shared->secondary_buckets);
+ hashtable->shared->secondary_buckets = InvalidDsaPointer;
+ }
+
+
+ /* This isn't a real phase: it's "past the end". */
+ /*
+ elog(LOG, "XXX ExecHashTableDestroy nbatch = %d", hashtable->curbatch);
+ elog(LOG, "XXX ExecHashTableDestroy expected %d got %d", BarrierPhase(barrier), PHJ_PHASE_PROMOTING_BATCH(hashtable->curbatch + 1));
+ Assert(BarrierPhase(barrier) ==
+ PHJ_PHASE_PROMOTING_BATCH(hashtable->curbatch + 1));
+ */
+ /* TODO: reinitialize barrier for rescan! */
+ /* TODO: free chunks? */
+ }
+ BarrierDetach(&hashtable->shared->barrier);
+ }
+
/*
* Make sure all the temp files are closed. We skip batch 0, since it
* can't have any temp files (and the arrays might not even exist if
@@ -600,6 +973,18 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
long nfreed;
HashMemoryChunk oldchunks;
+ /*
+ * TODO:TM this will be done incrementally for shared tables; for now it
+ * is disabled! Current idea: the chain of memory chunks can be shifted
+ * to another list of memory chunks to be rebatched, and other workers
+ * that are busy hashing can see that it's non-empty, and pop chunks off
+ * to rebatch. This way we can fan out the expensive rebatching work, but
+ * potentially requires more than one hash table active at a time. More
+ * study required.
+ */
+ if (HashJoinTableIsShared(hashtable))
+ return;
+
/* do nothing if we've decided to shut off growth */
if (!hashtable->growEnabled)
return;
@@ -670,13 +1055,13 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
* already been processed. We will free the old chunks as we go.
*/
memset(hashtable->buckets, 0, sizeof(HashJoinTuple) * hashtable->nbuckets);
- oldchunks = hashtable->chunks;
- hashtable->chunks = NULL;
+ oldchunks = hashtable->primary_chunk;
+ hashtable->primary_chunk = NULL;
/* so, let's scan through the old chunks, and all tuples in each chunk */
while (oldchunks != NULL)
{
- HashMemoryChunk nextchunk = oldchunks->next;
+ HashMemoryChunk nextchunk = oldchunks->next.private;
/* position within the buffer (up to oldchunks->used) */
size_t idx = 0;
@@ -699,20 +1084,23 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
/* keep tuple in memory - copy it into the new chunk */
HashJoinTuple copyTuple;
- copyTuple = (HashJoinTuple) dense_alloc(hashtable, hashTupleSize);
+ copyTuple = (HashJoinTuple)
+ dense_alloc(hashtable, hashTupleSize);
memcpy(copyTuple, hashTuple, hashTupleSize);
/* and add it back to the appropriate bucket */
- copyTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = copyTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, copyTuple,
+ InvalidDsaPointer);
}
else
{
/* dump it out */
Assert(batchno > curbatch);
- ExecHashJoinSaveTuple(HJTUPLE_MINTUPLE(hashTuple),
+ ExecHashJoinSaveTuple(hashtable,
+ HJTUPLE_MINTUPLE(hashTuple),
hashTuple->hashvalue,
- &hashtable->innerBatchFile[batchno]);
+ batchno,
+ true);
hashtable->spaceUsed -= hashTupleSize;
nfreed++;
@@ -758,8 +1146,6 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
static void
ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
{
- HashMemoryChunk chunk;
-
/* do nothing if not an increase (it's called increase for a reason) */
if (hashtable->nbuckets >= hashtable->nbuckets_optimal)
return;
@@ -780,16 +1166,156 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
* Just reallocate the proper number of buckets - we don't need to walk
* through them - we can walk the dense-allocated chunks (just like in
* ExecHashIncreaseNumBatches, but without all the copying into new
- * chunks)
+ * chunks): see ExecHashRebucket, which must be called next.
+ */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Size bytes;
+ int i;
+
+ /* Serial phase: only one backend reallocates. */
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_RESIZING);
+
+ /* Free the old arrays. */
+ dsa_free(hashtable->area,
+ hashtable->shared->primary_buckets);
+ dsa_free(hashtable->area,
+ hashtable->shared->secondary_buckets);
+ /* Allocate replacements. */
+ bytes = hashtable->nbuckets * sizeof(HashJoinBucketHead);
+ hashtable->shared->primary_buckets =
+ dsa_allocate(hashtable->area, bytes);
+ hashtable->shared->secondary_buckets =
+ dsa_allocate(hashtable->area, bytes);
+ if (!DsaPointerIsValid(hashtable->shared->primary_buckets) ||
+ !DsaPointerIsValid(hashtable->shared->secondary_buckets))
+ ereport(ERROR,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("out of memory")));
+ /* Initialize empty buckets. */
+ hashtable->buckets =
+ dsa_get_address(hashtable->area,
+ hashtable->shared->primary_buckets);
+ for (i = 0; i < hashtable->nbuckets; ++i)
+ dsa_pointer_atomic_write(&hashtable->buckets[i].shared,
+ InvalidDsaPointer);
+ hashtable->next_buckets =
+ dsa_get_address(hashtable->area,
+ hashtable->shared->secondary_buckets);
+ for (i = 0; i < hashtable->nbuckets; ++i)
+ dsa_pointer_atomic_write(&hashtable->next_buckets[i].shared,
+ InvalidDsaPointer);
+ hashtable->shared->nbuckets = hashtable->nbuckets;
+ /* Move all primary chunks to the rebucket list. */
+ dsa_pointer_atomic_write(&hashtable->shared->chunks_to_rebucket,
+ dsa_pointer_atomic_read(&hashtable->shared->head_primary_chunk));
+ dsa_pointer_atomic_write(&hashtable->shared->head_primary_chunk,
+ InvalidDsaPointer);
+ }
+ else
+ {
+ hashtable->buckets =
+ (HashJoinBucketHead *) repalloc(hashtable->buckets,
+ hashtable->nbuckets * sizeof(HashJoinBucketHead));
+
+ memset(hashtable->buckets, 0, hashtable->nbuckets * sizeof(HashJoinBucketHead));
+ /* Move all chunks to the rebucket list. */
+ hashtable->chunks_to_rebucket = hashtable->primary_chunk;
+ hashtable->primary_chunk = NULL;
+ }
+}
+
+/*
+ * Pop a memory chunk from a given list atomically. Returns a backend-local
+ * pointer to the chunk, or NULL if the list is empty. Also sets *chunk_out
+ * to the dsa_pointer to the chunk.
+ */
+static HashMemoryChunk
+ExecHashPopChunk(HashJoinTable hashtable,
+ dsa_pointer *chunk_out,
+ dsa_pointer_atomic *head)
+{
+ HashMemoryChunk chunk = NULL;
+
+ /*
+ * We could see a stale empty list and exist early without a barrier, so
+ * explicitly include one before we read the head of the list for the
+ * first time.
*/
- hashtable->buckets =
- (HashJoinTuple *) repalloc(hashtable->buckets,
- hashtable->nbuckets * sizeof(HashJoinTuple));
+ pg_read_barrier();
- memset(hashtable->buckets, 0, hashtable->nbuckets * sizeof(HashJoinTuple));
+ for (;;)
+ {
+ *chunk_out = dsa_pointer_atomic_read(head);
+ if (!DsaPointerIsValid(*chunk_out))
+ {
+ chunk = NULL;
+ break;
+ }
+ chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, *chunk_out);
+ if (dsa_pointer_atomic_compare_exchange(head,
+ chunk_out,
+ chunk->next.shared))
+ break;
+ }
- /* scan through all tuples in all chunks to rebuild the hash table */
- for (chunk = hashtable->chunks; chunk != NULL; chunk = chunk->next)
+ return chunk;
+}
+
+/*
+ * Push a shared memory chunk onto a given list atomically.
+ */
+static void
+ExecHashPushChunk(HashJoinTable hashtable,
+ HashMemoryChunk chunk,
+ dsa_pointer chunk_shared,
+ dsa_pointer_atomic *head)
+{
+ Assert(chunk == dsa_get_address(hashtable->area, chunk_shared));
+
+ for (;;)
+ {
+ chunk->next.shared = dsa_pointer_atomic_read(head);
+ if (dsa_pointer_atomic_compare_exchange(head,
+ &chunk->next.shared,
+ chunk_shared))
+ break;
+ }
+}
+
+/*
+ * ExecHashRebucket
+ * insert the tuples from all chunks into the correct bucket
+ */
+static void
+ExecHashRebucket(HashJoinTable hashtable)
+{
+ HashMemoryChunk chunk;
+ dsa_pointer chunk_shared;
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * This is a parallel phase. Workers will atomically pop one chunk at
+ * a time and rebucket all of its tuples.
+ */
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_REBUCKETING);
+ }
+
+ /*
+ * Scan through all tuples in all chunks in the rebucket list to rebuild
+ * the hash table.
+ */
+ if (HashJoinTableIsShared(hashtable))
+ chunk =
+ ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_to_rebucket);
+ else
+ chunk = hashtable->chunks_to_rebucket;
+ while (chunk != NULL)
{
/* process all tuples stored in this chunk */
size_t idx = 0;
@@ -797,6 +1323,8 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
while (idx < chunk->used)
{
HashJoinTuple hashTuple = (HashJoinTuple) (chunk->data + idx);
+ dsa_pointer hashTuple_shared = chunk_shared +
+ offsetof(HashMemoryChunkData, data) + idx;
int bucketno;
int batchno;
@@ -804,16 +1332,52 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
&bucketno, &batchno);
/* add the tuple to the proper bucket */
- hashTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = hashTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, hashTuple,
+ hashTuple_shared);
/* advance index past the tuple */
idx += MAXALIGN(HJTUPLE_OVERHEAD +
HJTUPLE_MINTUPLE(hashTuple)->t_len);
}
+
+ /* Push chunk onto regular list and move to next chunk. */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ &hashtable->shared->head_primary_chunk);
+ chunk =
+ ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_to_rebucket);
+ }
+ else
+ {
+ HashMemoryChunk next = chunk->next.private;
+
+ chunk->next.private = hashtable->primary_chunk;
+ hashtable->primary_chunk = chunk;
+ chunk = next;
+ }
}
}
+static void
+ExecHashTableComputeOptimalBuckets(HashJoinTable hashtable)
+{
+ double ntuples = (hashtable->totalTuples - hashtable->skewTuples);
+
+ /*
+ * Guard against integer overflow and alloc size overflow. The
+ * MaxAllocSize limitation doesn't really apply for shared hash tables,
+ * since DSA has no such limit, but for now let's apply the same limit.
+ */
+ while (ntuples > (hashtable->nbuckets_optimal * NTUP_PER_BUCKET) &&
+ hashtable->nbuckets_optimal <= INT_MAX / 2 &&
+ hashtable->nbuckets_optimal * 2 <= MaxAllocSize / sizeof(HashJoinBucketHead))
+ {
+ hashtable->nbuckets_optimal *= 2;
+ hashtable->log2_nbuckets_optimal += 1;
+ }
+}
/*
* ExecHashTableInsert
@@ -829,7 +1393,8 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
void
ExecHashTableInsert(HashJoinTable hashtable,
TupleTableSlot *slot,
- uint32 hashvalue)
+ uint32 hashvalue,
+ bool secondary)
{
MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot);
int bucketno;
@@ -848,11 +1413,17 @@ ExecHashTableInsert(HashJoinTable hashtable,
*/
HashJoinTuple hashTuple;
int hashTupleSize;
- double ntuples = (hashtable->totalTuples - hashtable->skewTuples);
+ dsa_pointer hashTuple_shared = InvalidDsaPointer;
/* Create the HashJoinTuple */
hashTupleSize = HJTUPLE_OVERHEAD + tuple->t_len;
- hashTuple = (HashJoinTuple) dense_alloc(hashtable, hashTupleSize);
+ if (HashJoinTableIsShared(hashtable))
+ hashTuple = (HashJoinTuple)
+ dense_alloc_shared(hashtable, hashTupleSize,
+ &hashTuple_shared, secondary);
+ else
+ hashTuple = (HashJoinTuple)
+ dense_alloc(hashtable, hashTupleSize);
hashTuple->hashvalue = hashvalue;
memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
@@ -866,25 +1437,16 @@ ExecHashTableInsert(HashJoinTable hashtable,
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple));
/* Push it onto the front of the bucket's list */
- hashTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = hashTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, hashTuple,
+ hashTuple_shared);
/*
* Increase the (optimal) number of buckets if we just exceeded the
* NTUP_PER_BUCKET threshold, but only when there's still a single
* batch.
*/
- if (hashtable->nbatch == 1 &&
- ntuples > (hashtable->nbuckets_optimal * NTUP_PER_BUCKET))
- {
- /* Guard against integer overflow and alloc size overflow */
- if (hashtable->nbuckets_optimal <= INT_MAX / 2 &&
- hashtable->nbuckets_optimal * 2 <= MaxAllocSize / sizeof(HashJoinTuple))
- {
- hashtable->nbuckets_optimal *= 2;
- hashtable->log2_nbuckets_optimal += 1;
- }
- }
+ if (hashtable->nbatch == 1)
+ ExecHashTableComputeOptimalBuckets(hashtable);
/* Account for space used, and back off if we've used too much */
hashtable->spaceUsed += hashTupleSize;
@@ -901,9 +1463,11 @@ ExecHashTableInsert(HashJoinTable hashtable,
* put the tuple into a temp file for later batches
*/
Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(tuple,
+ ExecHashJoinSaveTuple(hashtable,
+ tuple,
hashvalue,
- &hashtable->innerBatchFile[batchno]);
+ batchno,
+ true);
}
}
@@ -1047,6 +1611,138 @@ ExecHashGetBucketAndBatch(HashJoinTable hashtable,
}
/*
+ * Update the local hashtable with the current pointers and sizes from
+ * hashtable->parallel_state.
+ */
+void
+ExecHashUpdate(HashJoinTable hashtable)
+{
+ Barrier *barrier;
+
+ if (!HashJoinTableIsShared(hashtable))
+ return;
+
+ barrier = &hashtable->shared->barrier;
+
+ /*
+ * This should only be called in a phase when the hash table is not being
+ * mutated (ie resized, swapped etc).
+ */
+ Assert(!PHJ_PHASE_MUTATING_TABLE(
+ BarrierPhase(&hashtable->shared->barrier)));
+
+ /* The primary hash table. */
+ hashtable->buckets = (HashJoinBucketHead *)
+ dsa_get_address(hashtable->area,
+ hashtable->shared->primary_buckets);
+ hashtable->nbuckets = hashtable->shared->nbuckets;
+ hashtable->log2_nbuckets = my_log2(hashtable->nbuckets);
+ /* The secondary hash table, if there is one (NULL for initial batch). */
+ hashtable->next_buckets = (HashJoinBucketHead *)
+ dsa_get_address(hashtable->area,
+ hashtable->shared->secondary_buckets);
+
+ hashtable->curbatch = PHJ_PHASE_TO_BATCHNO(BarrierPhase(barrier));
+}
+
+/*
+ * Get the next tuple in the same bucket as 'tuple'.
+ */
+static HashJoinTuple
+next_tuple_in_bucket(HashJoinTable table, HashJoinTuple tuple)
+{
+ if (HashJoinTableIsShared(table))
+ return (HashJoinTuple)
+ dsa_get_address(table->area, tuple->next.shared);
+ else
+ return tuple->next.private;
+}
+
+/*
+ * Get the first tuple in a given skew bucket identified by number.
+ */
+static HashJoinTuple
+first_tuple_in_skew_bucket(HashJoinTable table, int skew_bucket_no)
+{
+ if (HashJoinTableIsShared(table))
+ return (HashJoinTuple)
+ dsa_get_address(table->area,
+ table->skewBucket[skew_bucket_no]->tuples.shared);
+ else
+ return table->skewBucket[skew_bucket_no]->tuples.private;
+}
+
+/*
+ * Get the first tuple in a given bucket identified by number.
+ */
+static HashJoinTuple
+first_tuple_in_bucket(HashJoinTable table, int bucket_no)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ dsa_pointer p =
+ dsa_pointer_atomic_read(&table->buckets[bucket_no].shared);
+ return (HashJoinTuple) dsa_get_address(table->area, p);
+ }
+ else
+ return table->buckets[bucket_no].private;
+}
+
+/*
+ * Insert a tuple at the front of a given bucket identified by number. For
+ * shared hash joins, tuple_shared must be provided, pointing to the tuple in
+ * the dsa_area backing the table. For private hash joins, it should be
+ * InvalidDsaPointer.
+ */
+static void
+insert_tuple_into_bucket(HashJoinTable table, int bucket_no,
+ HashJoinTuple tuple, dsa_pointer tuple_shared)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ Assert(tuple == dsa_get_address(table->area, tuple_shared));
+ for (;;)
+ {
+ tuple->next.shared =
+ dsa_pointer_atomic_read(&table->buckets[bucket_no].shared);
+ if (dsa_pointer_atomic_compare_exchange(&table->buckets[bucket_no].shared,
+ &tuple->next.shared,
+ tuple_shared))
+ break;
+ }
+ }
+ else
+ {
+ tuple->next.private = table->buckets[bucket_no].private;
+ table->buckets[bucket_no].private = tuple;
+ }
+}
+
+/*
+ * Insert a tuple at the front of a given skew bucket identified by number.
+ * For shared hash joins, tuple_shared must be provided, pointing to the tuple
+ * in the dsa_area backing the table. For private hash joins, it should be
+ * InvalidDsaPointer.
+ */
+static void
+insert_tuple_into_skew_bucket(HashJoinTable table, int skew_bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_shared)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ tuple->next.shared =
+ table->skewBucket[skew_bucket_no]->tuples.shared;
+ table->skewBucket[skew_bucket_no]->tuples.shared = tuple_shared;
+ }
+ else
+ {
+ tuple->next.private = table->skewBucket[skew_bucket_no]->tuples.private;
+ table->skewBucket[skew_bucket_no]->tuples.private = tuple;
+ }
+}
+
+/*
* ExecScanHashBucket
* scan a hash bucket for matches to the current outer tuple
*
@@ -1073,11 +1769,12 @@ ExecScanHashBucket(HashJoinState *hjstate,
* otherwise scan the standard hashtable bucket.
*/
if (hashTuple != NULL)
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
else if (hjstate->hj_CurSkewBucketNo != INVALID_SKEW_BUCKET_NO)
- hashTuple = hashtable->skewBucket[hjstate->hj_CurSkewBucketNo]->tuples;
+ hashTuple = first_tuple_in_skew_bucket(hashtable,
+ hjstate->hj_CurSkewBucketNo);
else
- hashTuple = hashtable->buckets[hjstate->hj_CurBucketNo];
+ hashTuple = first_tuple_in_bucket(hashtable, hjstate->hj_CurBucketNo);
while (hashTuple != NULL)
{
@@ -1101,7 +1798,7 @@ ExecScanHashBucket(HashJoinState *hjstate,
}
}
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
}
/*
@@ -1144,6 +1841,21 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
HashJoinTable hashtable = hjstate->hj_HashTable;
HashJoinTuple hashTuple = hjstate->hj_CurTuple;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ int phase PG_USED_FOR_ASSERTS_ONLY;
+
+ /*
+ * TODO: This walks the buckets in parallel mode, like the existing
+ * code, but it might make more sense to hand out chunks to workers
+ * instead of buckets.
+ */
+
+ phase = BarrierPhase(&hashtable->shared->barrier);
+ Assert(PHJ_PHASE_TO_SUBPHASE(phase) == PHJ_SUBPHASE_UNMATCHED);
+ Assert(PHJ_PHASE_TO_BATCHNO(phase) == hashtable->curbatch);
+ }
+
for (;;)
{
/*
@@ -1152,21 +1864,35 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
* bucket.
*/
if (hashTuple != NULL)
- hashTuple = hashTuple->next;
- else if (hjstate->hj_CurBucketNo < hashtable->nbuckets)
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
+ else if (HashJoinTableIsShared(hashtable))
{
- hashTuple = hashtable->buckets[hjstate->hj_CurBucketNo];
- hjstate->hj_CurBucketNo++;
+ int bucketno =
+ (int) pg_atomic_fetch_add_u32(
+ &hashtable->shared->next_unmatched_bucket, 1);
+
+ if (bucketno > hashtable->nbuckets)
+ break; /* finished all buckets */
+
+ hashTuple = first_tuple_in_bucket(hashtable, bucketno);
+
+ /* TODO: parallel skew bucket support */
}
- else if (hjstate->hj_CurSkewBucketNo < hashtable->nSkewBuckets)
+ else
{
- int j = hashtable->skewBucketNums[hjstate->hj_CurSkewBucketNo];
+ if (hjstate->hj_CurBucketNo < hashtable->nbuckets)
+ hashTuple = first_tuple_in_bucket(hashtable,
+ hjstate->hj_CurBucketNo++);
+ else if (hjstate->hj_CurSkewBucketNo < hashtable->nSkewBuckets)
+ {
+ int j = hashtable->skewBucketNums[hjstate->hj_CurSkewBucketNo];
- hashTuple = hashtable->skewBucket[j]->tuples;
- hjstate->hj_CurSkewBucketNo++;
+ hashTuple = first_tuple_in_skew_bucket(hashtable, j);
+ hjstate->hj_CurSkewBucketNo++;
+ }
+ else
+ break; /* finished all buckets */
}
- else
- break; /* finished all buckets */
while (hashTuple != NULL)
{
@@ -1191,7 +1917,7 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
return true;
}
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
}
}
@@ -1212,6 +1938,52 @@ ExecHashTableReset(HashJoinTable hashtable)
MemoryContext oldcxt;
int nbuckets = hashtable->nbuckets;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Wait for all workers to finish accessing the primary hash table. */
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_UNMATCHED);
+ if (BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASH_UNMATCHED))
+ {
+ /* Serial phase: promote the secondary table to primary. */
+ dsa_pointer tmp;
+ int i;
+
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_PROMOTING);
+
+ /* Clear the old primary table. */
+ for (i = 0; i < nbuckets; ++i)
+ dsa_pointer_atomic_write(&hashtable->buckets[i].shared,
+ InvalidDsaPointer);
+
+ /* Swap the two tables. */
+ tmp = hashtable->shared->primary_buckets;
+ hashtable->shared->primary_buckets =
+ hashtable->shared->secondary_buckets;
+ hashtable->shared->secondary_buckets = tmp;
+
+ /* Swap the chunk lists. */
+ tmp = dsa_pointer_atomic_read(&hashtable->shared->head_primary_chunk);
+ dsa_pointer_atomic_write(&hashtable->shared->head_primary_chunk,
+ dsa_pointer_atomic_read(&hashtable->shared->head_secondary_chunk));
+ dsa_pointer_atomic_write(&hashtable->shared->head_secondary_chunk,
+ tmp);
+
+ /* TODO: Free the secondary chunks. */
+ /* TODO: Or put them on a freelist instead? */
+
+ pg_atomic_write_u32(&hashtable->shared->next_unmatched_bucket,
+ 0);
+ }
+ /* Wait again, so that all workers now have the new table. */
+ BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASH_PROMOTING);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_LOADING);
+ ExecHashUpdate(hashtable);
+ return;
+ }
+
/*
* Release all the hash buckets and tuples acquired in the prior pass, and
* reinitialize the context for a new pass.
@@ -1220,15 +1992,15 @@ ExecHashTableReset(HashJoinTable hashtable)
oldcxt = MemoryContextSwitchTo(hashtable->batchCxt);
/* Reallocate and reinitialize the hash bucket headers. */
- hashtable->buckets = (HashJoinTuple *)
- palloc0(nbuckets * sizeof(HashJoinTuple));
+ hashtable->buckets = (HashJoinBucketHead *)
+ palloc0(nbuckets * sizeof(HashJoinBucketHead));
hashtable->spaceUsed = 0;
MemoryContextSwitchTo(oldcxt);
/* Forget the chunks (the memory was freed by the context reset above). */
- hashtable->chunks = NULL;
+ hashtable->primary_chunk = NULL;
}
/*
@@ -1241,10 +2013,14 @@ ExecHashTableResetMatchFlags(HashJoinTable hashtable)
HashJoinTuple tuple;
int i;
+ /* TODO: share parallel reset work! coordinate! */
+
/* Reset all flags in the main table ... */
for (i = 0; i < hashtable->nbuckets; i++)
{
- for (tuple = hashtable->buckets[i]; tuple != NULL; tuple = tuple->next)
+ for (tuple = first_tuple_in_bucket(hashtable, i);
+ tuple != NULL;
+ next_tuple_in_bucket(hashtable, tuple))
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(tuple));
}
@@ -1252,9 +2028,10 @@ ExecHashTableResetMatchFlags(HashJoinTable hashtable)
for (i = 0; i < hashtable->nSkewBuckets; i++)
{
int j = hashtable->skewBucketNums[i];
- HashSkewBucket *skewBucket = hashtable->skewBucket[j];
- for (tuple = skewBucket->tuples; tuple != NULL; tuple = tuple->next)
+ for (tuple = first_tuple_in_skew_bucket(hashtable, j);
+ tuple != NULL;
+ tuple = next_tuple_in_bucket(hashtable, tuple))
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(tuple));
}
}
@@ -1414,11 +2191,11 @@ ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node, int mcvsToUse)
continue;
/* Okay, create a new skew bucket for this hashvalue. */
- hashtable->skewBucket[bucket] = (HashSkewBucket *)
+ hashtable->skewBucket[bucket] = (HashSkewBucket *) /* TODO */
MemoryContextAlloc(hashtable->batchCxt,
sizeof(HashSkewBucket));
hashtable->skewBucket[bucket]->hashvalue = hashvalue;
- hashtable->skewBucket[bucket]->tuples = NULL;
+ hashtable->skewBucket[bucket]->tuples.private = NULL;
hashtable->skewBucketNums[hashtable->nSkewBuckets] = bucket;
hashtable->nSkewBuckets++;
hashtable->spaceUsed += SKEW_BUCKET_OVERHEAD;
@@ -1496,18 +2273,29 @@ ExecHashSkewTableInsert(HashJoinTable hashtable,
MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot);
HashJoinTuple hashTuple;
int hashTupleSize;
+ dsa_pointer tuple_pointer;
/* Create the HashJoinTuple */
hashTupleSize = HJTUPLE_OVERHEAD + tuple->t_len;
- hashTuple = (HashJoinTuple) MemoryContextAlloc(hashtable->batchCxt,
- hashTupleSize);
+ if (HashJoinTableIsShared(hashtable))
+ {
+ tuple_pointer = dsa_allocate(hashtable->area, hashTupleSize);
+ hashTuple = (HashJoinTuple) dsa_get_address(hashtable->area,
+ tuple_pointer);
+ }
+ else
+ {
+ tuple_pointer = InvalidDsaPointer;
+ hashTuple = (HashJoinTuple) MemoryContextAlloc(hashtable->batchCxt,
+ hashTupleSize);
+ }
hashTuple->hashvalue = hashvalue;
memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple));
/* Push it onto the front of the skew bucket's list */
- hashTuple->next = hashtable->skewBucket[bucketNumber]->tuples;
- hashtable->skewBucket[bucketNumber]->tuples = hashTuple;
+ insert_tuple_into_skew_bucket(hashtable, bucketNumber, hashTuple,
+ tuple_pointer);
/* Account for space used, and back off if we've used too much */
hashtable->spaceUsed += hashTupleSize;
@@ -1538,6 +2326,9 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
int batchno;
HashJoinTuple hashTuple;
+ /* TODO: skew buckets not yet supported for parallel mode */
+ Assert(!HashJoinTableIsShared(hashtable));
+
/* Locate the bucket to remove */
bucketToRemove = hashtable->skewBucketNums[hashtable->nSkewBuckets - 1];
bucket = hashtable->skewBucket[bucketToRemove];
@@ -1552,10 +2343,10 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno);
/* Process all tuples in the bucket */
- hashTuple = bucket->tuples;
+ hashTuple = first_tuple_in_skew_bucket(hashtable, bucketToRemove);
while (hashTuple != NULL)
{
- HashJoinTuple nextHashTuple = hashTuple->next;
+ HashJoinTuple nextHashTuple = next_tuple_in_bucket(hashtable, hashTuple);
MinimalTuple tuple;
Size tupleSize;
@@ -1581,8 +2372,8 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
memcpy(copyTuple, hashTuple, tupleSize);
pfree(hashTuple);
- copyTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = copyTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, copyTuple,
+ InvalidDsaPointer);
/* We have reduced skew space, but overall space doesn't change */
hashtable->spaceUsedSkew -= tupleSize;
@@ -1591,9 +2382,9 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
{
/* Put the tuple into a temp file for later batches */
Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(tuple, hashvalue,
- &hashtable->innerBatchFile[batchno]);
- pfree(hashTuple);
+ ExecHashJoinSaveTuple(hashtable, tuple, hashvalue,
+ batchno, true);
+ // pfree(hashTuple); /* TODO:TM */
hashtable->spaceUsed -= tupleSize;
hashtable->spaceUsedSkew -= tupleSize;
}
@@ -1636,6 +2427,173 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
}
/*
+ * For parallel execution, load as much of the next batch as we can as part of
+ * the probing phase for the current batch. This overlapping means that we do
+ * something useful before we start waiting for other workers.
+ */
+void
+ExecHashPreloadNextBatch(HashJoinTable hashtable)
+{
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier PG_USED_FOR_ASSERTS_ONLY = &hashtable->shared->barrier;
+ int curbatch = hashtable->curbatch;
+ int next_batch = curbatch + 1;
+
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_PROBING_BATCH(curbatch));
+
+ /* Prepare to read shared batch files for the next batch. */
+ ExecHashJoinInitializeBatchReader(hashtable, next_batch, true);
+
+ if (next_batch < hashtable->nbatch &&
+ hashtable->innerBatchFile[next_batch] != NULL)
+ {
+ /* TODO: Load into secondary hash table while memory is free! */
+ }
+
+ /*
+ * TODO: While doing this, also watch for chunks that can be
+ * rebatched, and help with that.
+ */
+ }
+}
+
+/*
+ * Add to the primary or secondary tuple counter.
+ */
+static void
+add_tuple_count(HashJoinTable hashtable, int count, bool secondary)
+{
+ if (secondary)
+ pg_atomic_fetch_add_u64(&hashtable->shared->total_secondary_tuples,
+ count);
+ else
+ {
+ uint32 total =
+ pg_atomic_fetch_add_u64(&hashtable->shared->total_primary_tuples,
+ count);
+ /* Also update this backend's counter. */
+ hashtable->totalTuples = total + count;
+ }
+}
+
+/*
+ * Allocate 'size' bytes from the currently active shared HashMemoryChunk.
+ * This is essentially the same as the private memory version, but allocates
+ * from separate chunks for the secondary table and periodically updates the
+ * shared tuple counter.
+ */
+static void *
+dense_alloc_shared(HashJoinTable hashtable,
+ Size size,
+ dsa_pointer *shared,
+ bool secondary)
+{
+ dsa_pointer chunk_shared;
+ HashMemoryChunk chunk;
+ char *ptr;
+
+ /* just in case the size is not already aligned properly */
+ size = MAXALIGN(size);
+
+ /*
+ * If tuple size is larger than of 1/4 of chunk size, allocate a separate
+ * chunk.
+ */
+ if (size > HASH_CHUNK_THRESHOLD)
+ {
+ /* allocate new chunk */
+ chunk_shared =
+ dsa_allocate(hashtable->area,
+ offsetof(HashMemoryChunkData, data) + size);
+ chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, chunk_shared);
+ *shared = chunk_shared + offsetof(HashMemoryChunkData, data);
+ chunk->maxlen = size;
+ chunk->used = size;
+ chunk->ntuples = 1;
+
+ /*
+ * Push onto the appropriate chunk list, but don't make it the current
+ * chunk because it hasn't got any more useful space in it. The
+ * current chunk may still have space, so keep that one current.
+ */
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ secondary ?
+ &hashtable->shared->head_secondary_chunk :
+ &hashtable->shared->head_primary_chunk);
+
+ /* Count these huge tuples immediately. */
+ add_tuple_count(hashtable, 1, secondary);
+ return chunk->data;
+ }
+
+ /*
+ * See if we have enough space for it in the current chunk (if any). If
+ * not, allocate a fresh chunk.
+ */
+ chunk = secondary ? hashtable->secondary_chunk : hashtable->primary_chunk;
+ if (chunk == NULL || (chunk->maxlen - chunk->used) < size)
+ {
+ /*
+ * Add the tuplecount for the outgoing chunk to the shared counter.
+ * Doing this only every time we need to allocate a new chunk should
+ * reduce contention on the shared counter.
+ */
+ if (chunk != NULL)
+ add_tuple_count(hashtable, chunk->ntuples, secondary);
+
+ /*
+ * Allocate new chunk and make it the current chunk for this backend
+ * to allocate from.
+ */
+ chunk_shared =
+ dsa_allocate(hashtable->area,
+ offsetof(HashMemoryChunkData, data) +
+ HASH_CHUNK_SIZE);
+ chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, chunk_shared);
+ *shared = chunk_shared + offsetof(HashMemoryChunkData, data);
+ if (secondary)
+ {
+ hashtable->secondary_chunk = chunk;
+ hashtable->secondary_chunk_shared = chunk_shared;
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ &hashtable->shared->head_secondary_chunk);
+ }
+ else
+ {
+ hashtable->primary_chunk = chunk;
+ hashtable->primary_chunk_shared = chunk_shared;
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ &hashtable->shared->head_primary_chunk);
+ }
+ chunk->maxlen = HASH_CHUNK_SIZE;
+ chunk->used = size;
+ chunk->ntuples = 1;
+
+ /*
+ * The shared tuple counter will be updated when this chunk is
+ * eventually full. See above.
+ */
+
+ return chunk->data;
+ }
+
+ /* There is enough space in the current chunk, let's add the tuple */
+ chunk_shared =
+ secondary ? hashtable->secondary_chunk_shared :
+ hashtable->primary_chunk_shared;
+ ptr = chunk->data + chunk->used;
+ *shared = chunk_shared + offsetof(HashMemoryChunkData, data) + chunk->used;
+ chunk->used += size;
+ chunk->ntuples += 1;
+
+ /* return pointer to the start of the tuple memory */
+ return ptr;
+}
+
+/*
* Allocate 'size' bytes from the currently active HashMemoryChunk
*/
static void *
@@ -1653,9 +2611,11 @@ dense_alloc(HashJoinTable hashtable, Size size)
*/
if (size > HASH_CHUNK_THRESHOLD)
{
+
/* allocate new chunk and put it at the beginning of the list */
- newChunk = (HashMemoryChunk) MemoryContextAlloc(hashtable->batchCxt,
- offsetof(HashMemoryChunkData, data) + size);
+ newChunk = (HashMemoryChunk)
+ MemoryContextAlloc(hashtable->batchCxt,
+ offsetof(HashMemoryChunkData, data) + size);
newChunk->maxlen = size;
newChunk->used = 0;
newChunk->ntuples = 0;
@@ -1664,15 +2624,15 @@ dense_alloc(HashJoinTable hashtable, Size size)
* Add this chunk to the list after the first existing chunk, so that
* we don't lose the remaining space in the "current" chunk.
*/
- if (hashtable->chunks != NULL)
+ if (hashtable->primary_chunk != NULL)
{
- newChunk->next = hashtable->chunks->next;
- hashtable->chunks->next = newChunk;
+ newChunk->next.private = hashtable->primary_chunk->next.private;
+ hashtable->primary_chunk->next.private = newChunk;
}
else
{
- newChunk->next = hashtable->chunks;
- hashtable->chunks = newChunk;
+ newChunk->next.private = NULL;
+ hashtable->primary_chunk = newChunk;
}
newChunk->used += size;
@@ -1685,27 +2645,27 @@ dense_alloc(HashJoinTable hashtable, Size size)
* See if we have enough space for it in the current chunk (if any). If
* not, allocate a fresh chunk.
*/
- if ((hashtable->chunks == NULL) ||
- (hashtable->chunks->maxlen - hashtable->chunks->used) < size)
+ if ((hashtable->primary_chunk == NULL) ||
+ (hashtable->primary_chunk->maxlen - hashtable->primary_chunk->used) < size)
{
/* allocate new chunk and put it at the beginning of the list */
- newChunk = (HashMemoryChunk) MemoryContextAlloc(hashtable->batchCxt,
- offsetof(HashMemoryChunkData, data) + HASH_CHUNK_SIZE);
-
+ newChunk = (HashMemoryChunk)
+ MemoryContextAlloc(hashtable->batchCxt,
+ offsetof(HashMemoryChunkData, data) +
+ HASH_CHUNK_SIZE);
+ newChunk->next.private = hashtable->primary_chunk;
+ hashtable->primary_chunk = newChunk;
newChunk->maxlen = HASH_CHUNK_SIZE;
newChunk->used = size;
newChunk->ntuples = 1;
- newChunk->next = hashtable->chunks;
- hashtable->chunks = newChunk;
-
return newChunk->data;
}
/* There is enough space in the current chunk, let's add the tuple */
- ptr = hashtable->chunks->data + hashtable->chunks->used;
- hashtable->chunks->used += size;
- hashtable->chunks->ntuples += 1;
+ ptr = hashtable->primary_chunk->data + hashtable->primary_chunk->used;
+ hashtable->primary_chunk->used += size;
+ hashtable->primary_chunk->ntuples += 1;
/* return pointer to the start of the tuple memory */
return ptr;
diff --git a/src/backend/executor/nodeHashjoin.c b/src/backend/executor/nodeHashjoin.c
index 369e666..3819151 100644
--- a/src/backend/executor/nodeHashjoin.c
+++ b/src/backend/executor/nodeHashjoin.c
@@ -21,8 +21,11 @@
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
#include "miscadmin.h"
+#include "pgstat.h"
+#include "storage/barrier.h"
#include "utils/memutils.h"
+#include <unistd.h> /* TODO: remove */
/*
* States of the ExecHashJoin state machine
@@ -46,7 +49,14 @@ static TupleTableSlot *ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
BufFile *file,
uint32 *hashvalue,
TupleTableSlot *tupleSlot);
+static TupleTableSlot *ExecHashJoinGetSavedTupleShared(HashJoinTable hashtable,
+ bool inner,
+ uint32 batchno,
+ uint32 *hashvalue,
+ TupleTableSlot *tupleSlot);
static bool ExecHashJoinNewBatch(HashJoinState *hjstate);
+static void ExecHashJoinLoadBatch(HashJoinState *hjstate);
+static void ExecHashJoinExportBatches(HashJoinTable hashtable);
/* ----------------------------------------------------------------
@@ -147,6 +157,14 @@ ExecHashJoin(HashJoinState *node)
/* no chance to not build the hash table */
node->hj_FirstOuterTupleSlot = NULL;
}
+ else if (hashNode->shared_table_data != NULL)
+ {
+ /*
+ * TODO: The empty-outer optimization is not implemented
+ * for shared hash tables yet.
+ */
+ node->hj_FirstOuterTupleSlot = NULL;
+ }
else if (HJ_FILL_OUTER(node) ||
(outerNode->plan->startup_cost < hashNode->ps.plan->total_cost &&
!node->hj_OuterNotEmpty))
@@ -166,7 +184,7 @@ ExecHashJoin(HashJoinState *node)
/*
* create the hash table
*/
- hashtable = ExecHashTableCreate((Hash *) hashNode->ps.plan,
+ hashtable = ExecHashTableCreate(hashNode,
node->hj_HashOperators,
HJ_FILL_INNER(node));
node->hj_HashTable = hashtable;
@@ -177,12 +195,29 @@ ExecHashJoin(HashJoinState *node)
hashNode->hashtable = hashtable;
(void) MultiExecProcNode((PlanState *) hashNode);
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(&hashtable->shared->barrier) >=
+ PHJ_PHASE_HASHING);
+
+ /* Allow other backends to access batches we generated. */
+ ExecHashJoinExportBatches(hashtable);
+
+ /*
+ * Check if we are a worker that attached too late to
+ * avoid deadlock risk with the leader.
+ */
+ if (ExecHashCheckForEarlyExit(hashtable))
+ return NULL;
+ }
+
/*
* If the inner relation is completely empty, and we're not
* doing a left outer join, we can quit without scanning the
* outer relation.
*/
- if (hashtable->totalTuples == 0 && !HJ_FILL_OUTER(node))
+ if (!HashJoinTableIsShared(hashtable) && /* TODO:TM */
+ hashtable->totalTuples == 0 && !HJ_FILL_OUTER(node))
return NULL;
/*
@@ -198,12 +233,66 @@ ExecHashJoin(HashJoinState *node)
*/
node->hj_OuterNotEmpty = false;
- node->hj_JoinState = HJ_NEED_NEW_OUTER;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier = &hashtable->shared->barrier;
+ int phase = BarrierPhase(barrier);
+
+ /*
+ * Map the current phase to the appropriate initial state
+ * for this worker, so we can get started.
+ */
+ Assert(BarrierPhase(barrier) >= PHJ_PHASE_PROBING);
+ hashtable->curbatch = PHJ_PHASE_TO_BATCHNO(phase);
+ switch (PHJ_PHASE_TO_SUBPHASE(phase))
+ {
+ case PHJ_SUBPHASE_PROMOTING:
+ /* Wait for serial phase to finish. */
+ BarrierWait(barrier, WAIT_EVENT_HASHJOIN_PROMOTING);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(barrier)) ==
+ PHJ_SUBPHASE_LOADING);
+ /* fall through */
+ case PHJ_SUBPHASE_LOADING:
+ /* Help load the current batch. */
+ ExecHashUpdate(hashtable);
+ ExecHashJoinInitializeBatchReader(hashtable,
+ hashtable->curbatch,
+ true);
+ ExecHashJoinLoadBatch(node);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(barrier)) ==
+ PHJ_SUBPHASE_PROBING);
+ /* fall through */
+ case PHJ_SUBPHASE_PROBING:
+ /* Help probe the current batch. */
+ ExecHashUpdate(hashtable);
+ ExecHashJoinInitializeBatchReader(hashtable,
+ hashtable->curbatch,
+ false);
+ node->hj_JoinState = HJ_NEED_NEW_OUTER;
+ break;
+ case PHJ_SUBPHASE_UNMATCHED:
+ /* Help scan for unmatched inner tuples. */
+ ExecHashUpdate(hashtable);
+ node->hj_JoinState = HJ_FILL_INNER_TUPLES;
+ break;
+ }
+ continue;
+ }
+ else
+ node->hj_JoinState = HJ_NEED_NEW_OUTER;
/* FALL THRU */
case HJ_NEED_NEW_OUTER:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(PHJ_PHASE_TO_BATCHNO(BarrierPhase(&hashtable->shared->barrier)) ==
+ hashtable->curbatch);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_PROBING);
+ }
+
/*
* We don't have an outer tuple, try to get the next one
*/
@@ -213,6 +302,38 @@ ExecHashJoin(HashJoinState *node)
if (TupIsNull(outerTupleSlot))
{
/* end of batch, or maybe whole join */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Allow other backends to access our batches. */
+ ExecHashJoinExportBatches(hashtable);
+ /*
+ * Check if we are a leader that can't go further than
+ * probing the first batch without deadlock risk,
+ * because there are workers running.
+ */
+ if (ExecHashCheckForEarlyExit(hashtable))
+ {
+ elog(LOG, "leader detaching!");
+ return NULL;
+ }
+
+ /*
+ * We may be able to load some amount of the next
+ * batch into spare work_mem, before we start waiting
+ * for other workers to finish probing the current
+ * batch.
+ */
+ ExecHashPreloadNextBatch(hashtable);
+ /*
+ * You can't start searching for unmatched tuples
+ * until all workers have finished probing, so we
+ * synchronize here.
+ */
+ BarrierWait(&hashtable->shared->barrier,
+ WAIT_EVENT_HASHJOIN_PROBING);
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_UNMATCHED_BATCH(hashtable->curbatch));
+ }
if (HJ_FILL_INNER(node))
{
/* set up to scan for unmatched inner tuples */
@@ -250,9 +371,9 @@ ExecHashJoin(HashJoinState *node)
* Save it in the corresponding outer-batch file.
*/
Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(ExecFetchSlotMinimalTuple(outerTupleSlot),
- hashvalue,
- &hashtable->outerBatchFile[batchno]);
+ ExecHashJoinSaveTuple(hashtable,
+ ExecFetchSlotMinimalTuple(outerTupleSlot),
+ hashvalue, batchno, false);
/* Loop around, staying in HJ_NEED_NEW_OUTER state */
continue;
}
@@ -296,6 +417,13 @@ ExecHashJoin(HashJoinState *node)
if (joinqual == NIL || ExecQual(joinqual, econtext, false))
{
node->hj_MatchedOuter = true;
+ /*
+ * Note: it is OK to do this in a shared hash table
+ * without any kind of memory synchronization, because the
+ * only transition is 0->1, so ordering doesn't matter if
+ * several backends do it, and there will be a memory
+ * barrier before anyone reads it.
+ */
HeapTupleHeaderSetMatch(HJTUPLE_MINTUPLE(node->hj_CurTuple));
/* In an antijoin, we never return a matched tuple */
@@ -702,10 +830,18 @@ ExecHashJoinOuterGetTuple(PlanState *outerNode,
if (file == NULL)
return NULL;
- slot = ExecHashJoinGetSavedTuple(hjstate,
- file,
- hashvalue,
- hjstate->hj_OuterTupleSlot);
+ /* TODO: refactor to one function call? */
+ if (HashJoinTableIsShared(hashtable))
+ slot = ExecHashJoinGetSavedTupleShared(hashtable,
+ false,
+ curbatch,
+ hashvalue,
+ hjstate->hj_OuterTupleSlot);
+ else
+ slot = ExecHashJoinGetSavedTuple(hjstate,
+ file,
+ hashvalue,
+ hjstate->hj_OuterTupleSlot);
if (!TupIsNull(slot))
return slot;
}
@@ -726,13 +862,17 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
HashJoinTable hashtable = hjstate->hj_HashTable;
int nbatch;
int curbatch;
- BufFile *innerFile;
- TupleTableSlot *slot;
- uint32 hashvalue;
+ Barrier *barrier;
nbatch = hashtable->nbatch;
curbatch = hashtable->curbatch;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ barrier = &hashtable->shared->barrier;
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_UNMATCHED_BATCH(curbatch));
+ }
+
if (curbatch > 0)
{
/*
@@ -793,6 +933,20 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
nbatch != hashtable->nbatch_outstart)
break; /* must process due to rule 3 */
/* We can ignore this batch. */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Skip the batch, but stay in sync with group. */
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_UNMATCHED_BATCH(curbatch - 1));
+ ExecHashTableReset(hashtable);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_LOADING_BATCH(curbatch));
+ if (BarrierWait(&hashtable->shared->barrier,
+ WAIT_EVENT_HASHJOIN_SKIP_LOADING))
+ ExecHashJoinResetBatchReaders(hashtable);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_PROBING_BATCH(curbatch));
+ BarrierWait(&hashtable->shared->barrier,
+ WAIT_EVENT_HASHJOIN_SKIP_PROBING);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_UNMATCHED_BATCH(curbatch));
+ }
/* Release associated temp files right away. */
if (hashtable->innerBatchFile[curbatch])
BufFileClose(hashtable->innerBatchFile[curbatch]);
@@ -812,26 +966,63 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
* Reload the hash table with the new inner batch (which could be empty)
*/
ExecHashTableReset(hashtable);
+ ExecHashJoinLoadBatch(hjstate);
+
+ return true;
+}
+
+static void
+ExecHashJoinLoadBatch(HashJoinState *hjstate)
+{
+ HashJoinTable hashtable = hjstate->hj_HashTable;
+ int curbatch = hashtable->curbatch;
+ BufFile *innerFile;
+ TupleTableSlot *slot;
+ uint32 hashvalue;
+
+ if (HashJoinTableIsShared(hashtable))
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_LOADING);
innerFile = hashtable->innerBatchFile[curbatch];
if (innerFile != NULL)
{
- if (BufFileSeek(innerFile, 0, 0L, SEEK_SET))
- ereport(ERROR,
- (errcode_for_file_access(),
+ /*
+ * TODO: Do not rewind inner batch file for shared hash tables,
+ * because ExecHashPreloadNextBatch already did that and left the read
+ * head at the right place for us to continue. Tidy up...
+ */
+ if (!HashJoinTableIsShared(hashtable))
+ {
+ if (BufFileSeek(innerFile, 0, 0L, SEEK_SET))
+ ereport(ERROR,
+ (errcode_for_file_access(),
errmsg("could not rewind hash-join temporary file: %m")));
+ }
- while ((slot = ExecHashJoinGetSavedTuple(hjstate,
+ for (;;)
+ {
+ /* TODO: refactor this into one function call? */
+ if (HashJoinTableIsShared(hashtable))
+ slot = ExecHashJoinGetSavedTupleShared(hashtable,
+ true,
+ curbatch,
+ &hashvalue,
+ hjstate->hj_HashTupleSlot);
+ else
+ slot = ExecHashJoinGetSavedTuple(hjstate,
innerFile,
&hashvalue,
- hjstate->hj_HashTupleSlot)))
- {
+ hjstate->hj_HashTupleSlot);
+ if (slot == NULL)
+ break;
+
/*
* NOTE: some tuples may be sent to future batches. Also, it is
* possible for hashtable->nbatch to be increased here!
*/
- ExecHashTableInsert(hashtable, slot, hashvalue);
+ ExecHashTableInsert(hashtable, slot, hashvalue, false);
}
/*
@@ -845,7 +1036,7 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
/*
* Rewind outer batch file (if present), so that we can start reading it.
*/
- if (hashtable->outerBatchFile[curbatch] != NULL)
+ if (!HashJoinTableIsShared(hashtable) && hashtable->outerBatchFile[curbatch] != NULL)
{
if (BufFileSeek(hashtable->outerBatchFile[curbatch], 0, 0L, SEEK_SET))
ereport(ERROR,
@@ -853,7 +1044,112 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
errmsg("could not rewind hash-join temporary file: %m")));
}
- return true;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Wait until all workers have finished loading their portion of the
+ * hash table, so that all workers can start probing.
+ */
+ if (BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASHJOIN_LOADING))
+ ExecHashJoinResetBatchReaders(hashtable);
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_PROBING_BATCH(hashtable->curbatch));
+ ExecHashJoinInitializeBatchReader(hashtable, hashtable->curbatch, false);
+ }
+}
+
+/*
+ * Export a BufFile, copy the descriptor to DSA memory and return the
+ * dsa_pointer.
+ */
+static dsa_pointer
+make_batch_descriptor(dsa_area *area, BufFile *file)
+{
+ dsa_pointer pointer;
+ BufFileDescriptor *source;
+ BufFileDescriptor *target;
+ size_t size;
+
+ source = BufFileExport(file);
+ size = BufFileDescriptorSize(source);
+ pointer = dsa_allocate(area, size);
+ if (!DsaPointerIsValid(pointer))
+ ereport(ERROR,
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of memory"),
+ errdetail("Failed on dsa_allocate of size %zu.", size)));
+ target = dsa_get_address(area, pointer);
+ memcpy(target, source, size);
+ pfree(source);
+
+ return pointer;
+}
+
+/*
+ * Publish a batch descriptor for a future batch so that other participants
+ * can import it and read it. If 'descriptor' is InvalidDsaPointer, then
+ * forget the published descriptor so that it will be reexported later.
+ */
+static void
+set_batch_descriptor(HashJoinTable hashtable, int batchno, bool inner,
+ dsa_pointer descriptor)
+{
+ HashJoinParticipantState *participant;
+ dsa_pointer *level1;
+ dsa_pointer *level2;
+ int rank;
+ int index;
+
+ participant = &hashtable->shared->participants[HashJoinParticipantNumber()];
+ rank = fls(batchno);
+ index = batchno % (1 << (rank - 1));
+ level1 = inner ? participant->inner_batch_descriptors
+ : participant->outer_batch_descriptors;
+ if (level1[rank] == InvalidDsaPointer)
+ {
+ size_t size = sizeof(dsa_pointer) * (1 << rank);
+
+ level1[rank] = dsa_allocate(hashtable->area, size);
+ if (level1[rank] == InvalidDsaPointer)
+ ereport(ERROR,
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of memory"),
+ errdetail("Failed on dsa_allocate of size %zu.", size)));
+ level2 = dsa_get_address(hashtable->area, level1[rank]);
+ memset(level2, 0, size);
+ }
+ level2 = dsa_get_address(hashtable->area, level1[rank]);
+ if (level2[index] != InvalidDsaPointer)
+ dsa_free(hashtable->area, level2[index]);
+ level2[index] = descriptor;
+}
+
+/*
+ * Get a batch descriptor published by a given participant, if there is one.
+ */
+static BufFileDescriptor *
+get_batch_descriptor(HashJoinTable hashtable, int participant_number,
+ int batchno, bool inner)
+{
+ HashJoinParticipantState *participant;
+ dsa_pointer *level1;
+ dsa_pointer *level2;
+ int rank;
+ int index;
+
+ participant = &hashtable->shared->participants[participant_number];
+ rank = fls(batchno);
+ index = batchno % (1 << (rank - 1));
+ level1 = inner ? participant->inner_batch_descriptors
+ : participant->outer_batch_descriptors;
+ if (level1[rank] == InvalidDsaPointer)
+ return NULL;
+ level2 = dsa_get_address(hashtable->area, level1[rank]);
+ if (level2[index] == InvalidDsaPointer)
+ return NULL;
+
+ return (BufFileDescriptor *)
+ dsa_get_address(hashtable->area, level2[index]);
}
/*
@@ -868,17 +1164,33 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
* will get messed up.
*/
void
-ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
- BufFile **fileptr)
+ExecHashJoinSaveTuple(HashJoinTable hashtable,
+ MinimalTuple tuple, uint32 hashvalue,
+ int batchno,
+ bool inner)
+ //BufFile **fileptr)
{
- BufFile *file = *fileptr;
+ BufFile *file;
size_t written;
+ if (inner)
+ file = hashtable->innerBatchFile[batchno];
+ else
+ file = hashtable->outerBatchFile[batchno];
if (file == NULL)
{
/* First write to this batch file, so open it. */
file = BufFileCreateTemp(false);
- *fileptr = file;
+ if (inner)
+ hashtable->innerBatchFile[batchno] = file;
+ else
+ hashtable->outerBatchFile[batchno] = file;
+ }
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* This batch needs to be re-exported, if it was already exported. */
+ set_batch_descriptor(hashtable, batchno, inner, InvalidDsaPointer);
}
written = BufFileWrite(file, (void *) &hashvalue, sizeof(uint32));
@@ -939,10 +1251,229 @@ ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
return ExecStoreMinimalTuple(tuple, tupleSlot, true);
}
+/*
+ * Export unexported future batches created by this participant, so that other
+ * participants can read from them after they have finished reading their own.
+ */
+static void
+ExecHashJoinExportBatches(HashJoinTable hashtable)
+{
+ int i;
+
+ /* Find this participant's HashJoinParticipantState object. */
+ Assert(HashJoinParticipantNumber() < hashtable->shared->planned_participants);
+
+ /* Export future batches and copy their descriptors into DSA memory. */
+ for (i = hashtable->curbatch + 1; i < hashtable->nbatch; ++i)
+ {
+ if (hashtable->innerBatchFile[i] != NULL &&
+ get_batch_descriptor(hashtable, HashJoinParticipantNumber(), i, true) == InvalidDsaPointer)
+ set_batch_descriptor(hashtable, i, true,
+ make_batch_descriptor(hashtable->area, hashtable->innerBatchFile[i]));
+ if (hashtable->outerBatchFile[i] != NULL &&
+ get_batch_descriptor(hashtable, HashJoinParticipantNumber(), i, false) == InvalidDsaPointer)
+ set_batch_descriptor(hashtable, i, true,
+ make_batch_descriptor(hashtable->area, hashtable->outerBatchFile[i]));
+ }
+}
+
+/*
+ * Initialize the batch reader to prepare it for reading a given batch.
+ */
+void
+ExecHashJoinInitializeBatchReader(HashJoinTable hashtable,
+ int batchno,
+ bool inner)
+{
+ HashJoinBatchReader *batch_reader;
+ HashJoinParticipantState *participant;
+
+ batch_reader = &hashtable->batch_reader;
+
+ if (!HashJoinTableIsShared(hashtable))
+ return;
+ if (hashtable->nbatch <= 1)
+ return;
+
+ /* We always start reading from the batch file that this backend wrote. */
+ batch_reader->participant_number = HashJoinParticipantNumber();
+ batch_reader->head.fileno = batch_reader->head.offset = -1;
+ participant = &hashtable->shared->participants[HashJoinParticipantNumber()];
+ if (inner)
+ {
+ batch_reader->shared = &participant->inner_batch_reader;
+ batch_reader->file = hashtable->innerBatchFile[batchno];
+ }
+ else
+ {
+ batch_reader->shared = &participant->outer_batch_reader;
+ batch_reader->file = hashtable->outerBatchFile[batchno];
+ }
+}
+
+/*
+ * Reset the shared read heads on all shared batch file readers. Must
+ * be called only in one backend.
+ */
+void
+ExecHashJoinResetBatchReaders(HashJoinTable hashtable)
+{
+ int i;
+
+ for (i = 0; i < hashtable->shared->planned_participants; ++i)
+ {
+ hashtable->shared->participants[i].inner_batch_reader.head.fileno = 0;
+ hashtable->shared->participants[i].inner_batch_reader.head.offset = 0;
+ hashtable->shared->participants[i].outer_batch_reader.head.fileno = 0;
+ hashtable->shared->participants[i].outer_batch_reader.head.offset = 0;
+ }
+}
+
+/*
+ * ExecHashJoinGetSavedTupleShared
+ * read the next tuple from a batch file, including the batch files of
+ * other participants. Return NULL if no more.
+ *
+ * On success, *hashvalue is set to the tuple's hash value, and the tuple
+ * itself is stored in the given slot.
+ */
+static TupleTableSlot *
+ExecHashJoinGetSavedTupleShared(HashJoinTable hashtable,
+ bool inner,
+ uint32 batchno,
+ uint32 *hashvalue,
+ TupleTableSlot *tupleSlot)
+{
+ TupleTableSlot *result = NULL;
+ HashJoinBatchReader *batch_reader = &hashtable->batch_reader;
+ BufFileDescriptor *descriptor;
+
+ Assert(HashJoinTableIsShared(hashtable));
+
+ for (;;)
+ {
+ uint32 header[2];
+ size_t nread;
+ MinimalTuple tuple;
+
+ if (hashtable->batch_reader.file == NULL)
+ {
+ /*
+ * No file found for the current participant. Try stealing tuples
+ * from the next participant.
+ */
+ goto next_participant;
+ }
+
+ LWLockAcquire(&batch_reader->shared->lock, LW_EXCLUSIVE);
+ if (batch_reader->shared->error)
+ {
+ /* Don't try to read if reading failed in some other backend. */
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from hash-join temporary file")));
+ }
+
+ /* Set the shared error flag, which we'll clear if we succeed. */
+ batch_reader->shared->error = true;
+
+ /*
+ * If another worker has moved the shared read head since we last read,
+ * we'll need to seek to the new shared position.
+ */
+ if (batch_reader->head.fileno != batch_reader->shared->head.fileno ||
+ batch_reader->head.offset != batch_reader->shared->head.offset)
+ {
+ BufFileSeek(batch_reader->file,
+ batch_reader->shared->head.fileno,
+ batch_reader->shared->head.offset,
+ SEEK_SET);
+ batch_reader->head = batch_reader->shared->head;
+ }
+
+ /* Try to read the size and hash. */
+ nread = BufFileRead(batch_reader->file, (void *) header, sizeof(header));
+ if (nread > 0)
+ {
+ if (nread != sizeof(header))
+ {
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from hash-join temporary file: %m")));
+ }
+ *hashvalue = header[0];
+ tuple = (MinimalTuple) palloc(header[1]);
+ tuple->t_len = header[1];
+ nread = BufFileRead(batch_reader->file,
+ (void *) ((char *) tuple + sizeof(uint32)),
+ header[1] - sizeof(uint32));
+ if (nread != header[1] - sizeof(uint32))
+ {
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from hash-join temporary file: %m")));
+ }
+
+ result = ExecStoreMinimalTuple(tuple, tupleSlot, true);
+
+ }
+ /* Commit to shared memory. */
+ BufFileTell(batch_reader->file,
+ &batch_reader->head.fileno,
+ &batch_reader->head.offset);
+ batch_reader->shared->head = batch_reader->head;
+ batch_reader->shared->error = false;
+ LWLockRelease(&batch_reader->shared->lock);
+
+ if (result != NULL)
+ return result;
+
+next_participant:
+ /* Try the next participant's batch file. */
+ batch_reader->participant_number =
+ (batch_reader->participant_number + 1) %
+ hashtable->shared->planned_participants;
+ if (batch_reader->participant_number == HashJoinParticipantNumber())
+ {
+ /*
+ * We've made it all the way back to the file we started with,
+ * which is the one that this backend wrote. So there are no more
+ * tuples to be had in any participant's batch file.
+ */
+ ExecClearTuple(tupleSlot);
+ return NULL;
+ }
+
+ /* Import the BufFile from that participant, if it exported one. */
+ descriptor = get_batch_descriptor(hashtable,
+ batch_reader->participant_number,
+ batchno,
+ inner);
+ if (descriptor == NULL)
+ batch_reader->file = NULL;
+ else
+ batch_reader->file = BufFileImport(descriptor);
+ batch_reader->shared = inner ? &hashtable->shared->participants[batch_reader->participant_number].inner_batch_reader
+ : &hashtable->shared->participants[batch_reader->participant_number].outer_batch_reader;
+ batch_reader->head.fileno = batch_reader->head.offset = 0;
+ }
+}
void
ExecReScanHashJoin(HashJoinState *node)
{
+ HashState *hashNode = (HashState *) innerPlanState(node);
+
+ /* We can't use HashJoinTableIsShared if the table is NULL. */
+ if (hashNode->shared_table_data != NULL)
+ {
+ elog(ERROR, "TODO: ExecReScanHashJoin not working yet");
+
+ /* Coordinate a rewind to the shared hash table creation phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier, PHJ_PHASE_INIT,
+ WAIT_EVENT_HASHJOIN_REWINDING);
+ }
+
/*
* In a multi-batch join, we currently have to do rescans the hard way,
* primarily because batch temp files may have already been released. But
@@ -977,6 +1508,15 @@ ExecReScanHashJoin(HashJoinState *node)
/* ExecHashJoin can skip the BUILD_HASHTABLE step */
node->hj_JoinState = HJ_NEED_NEW_OUTER;
+
+ if (HashJoinTableIsShared(node->hj_HashTable))
+ {
+ /* Coordinate a rewind to the shared probing phase. */
+ if (BarrierWaitSet(&hashNode->shared_table_data->barrier,
+ PHJ_PHASE_PROBING,
+ WAIT_EVENT_HASHJOIN_REWINDING2))
+ ExecHashJoinResetBatchReaders(node->hj_HashTable);
+ }
}
else
{
@@ -985,6 +1525,14 @@ ExecReScanHashJoin(HashJoinState *node)
node->hj_HashTable = NULL;
node->hj_JoinState = HJ_BUILD_HASHTABLE;
+ if (HashJoinTableIsShared(node->hj_HashTable))
+ {
+ /* Coordinate a rewind to the shared hash table creation phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier,
+ PHJ_PHASE_INIT,
+ WAIT_EVENT_HASHJOIN_REWINDING3);
+ }
+
/*
* if chgParam of subnode is not null then plan will be re-scanned
* by first ExecProcNode.
@@ -1011,3 +1559,76 @@ ExecReScanHashJoin(HashJoinState *node)
if (node->js.ps.lefttree->chgParam == NULL)
ExecReScan(node->js.ps.lefttree);
}
+
+void ExecHashJoinEstimate(HashJoinState *state, ParallelContext *pcxt)
+{
+ size_t size;
+
+ size = offsetof(SharedHashJoinTableData, participants) +
+ sizeof(HashJoinParticipantState) * (pcxt->nworkers + 1);
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+void
+ExecHashJoinInitializeDSM(HashJoinState *state, ParallelContext *pcxt)
+{
+ HashState *hashNode;
+ SharedHashJoinTable shared;
+ size_t size;
+ int planned_participants;
+
+ /*
+ * Set up the state needed to coordinate access to the shared hash table,
+ * using the plan node ID as the toc key.
+ */
+ planned_participants = pcxt->nworkers + 1; /* possible workers + leader */
+ size = offsetof(SharedHashJoinTableData, participants) +
+ sizeof(HashJoinParticipantState) * planned_participants;
+ shared = shm_toc_allocate(pcxt->toc, size);
+ BarrierInit(&shared->barrier, 0);
+ shared->primary_buckets = InvalidDsaPointer;
+ shared->secondary_buckets = InvalidDsaPointer;
+ pg_atomic_init_u32(&shared->next_unmatched_bucket, 0);
+ pg_atomic_init_u64(&shared->total_primary_tuples, 0);
+ pg_atomic_init_u64(&shared->total_secondary_tuples, 0);
+ dsa_pointer_atomic_init(&shared->head_primary_chunk, InvalidDsaPointer);
+ dsa_pointer_atomic_init(&shared->head_secondary_chunk, InvalidDsaPointer);
+ dsa_pointer_atomic_init(&shared->chunks_to_rebucket, InvalidDsaPointer);
+ shared->planned_participants = planned_participants;
+ shm_toc_insert(pcxt->toc, state->js.ps.plan->plan_node_id, shared);
+
+ /*
+ * Pass the SharedHashJoinTable to the hash node. If the Gather node
+ * running in the leader backend decides to execute the hash join, it
+ * hasn't called ExecHashJoinInitializeWorker so it doesn't have
+ * state->shared_table_data set up. So we must do it here.
+ */
+ hashNode = (HashState *) innerPlanState(state);
+ hashNode->shared_table_data = shared;
+}
+
+void
+ExecHashJoinInitializeWorker(HashJoinState *state, shm_toc *toc)
+{
+ HashState *hashNode;
+
+ state->hj_sharedHashJoinTable =
+ shm_toc_lookup(toc, state->js.ps.plan->plan_node_id);
+
+ /*
+ * Inject SharedHashJoinTable into the hash node. It could instead have
+ * its own ExecHashInitializeWorker function, but we only want to set its
+ * 'parallel_aware' flag if we want to tell it to actually build the hash
+ * table in parallel. Since its parallel_aware flag also controls whether
+ * its 'InitializeWorker' function gets called, and it also needs access
+ * to this object for serial shared hash mode, we'll pass it on here
+ * instead of depending on that.
+ */
+ hashNode = (HashState *) innerPlanState(state);
+ hashNode->shared_table_data = state->hj_sharedHashJoinTable;
+ Assert(hashNode->shared_table_data != NULL);
+
+ Assert(HashJoinParticipantNumber() <
+ hashNode->shared_table_data->planned_participants);
+}
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 00bf3a5..361eb5d 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -31,6 +31,8 @@
#include "executor/nodeSeqscan.h"
#include "utils/rel.h"
+#include <unistd.h>
+
static void InitScanRelation(SeqScanState *node, EState *estate, int eflags);
static TupleTableSlot *SeqNext(SeqScanState *node);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index ae86954..ca215dd 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1993,6 +1993,7 @@ _outHashPath(StringInfo str, const HashPath *node)
WRITE_NODE_FIELD(path_hashclauses);
WRITE_INT_FIELD(num_batches);
+ WRITE_ENUM_FIELD(table_type, HashPathTableType);
}
static void
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..79c7650 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -104,6 +104,7 @@
double seq_page_cost = DEFAULT_SEQ_PAGE_COST;
double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
+double cpu_shared_tuple_cost = DEFAULT_CPU_SHARED_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
double parallel_tuple_cost = DEFAULT_PARALLEL_TUPLE_COST;
@@ -2694,7 +2695,8 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
List *hashclauses,
Path *outer_path, Path *inner_path,
SpecialJoinInfo *sjinfo,
- SemiAntiJoinFactors *semifactors)
+ SemiAntiJoinFactors *semifactors,
+ HashPathTableType table_type)
{
Cost startup_cost = 0;
Cost run_cost = 0;
@@ -2725,6 +2727,26 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
run_cost += cpu_operator_cost * num_hashclauses * outer_path_rows;
/*
+ * If this is a shared hash table, there is a extra charge for inserting
+ * each tuple into the shared hash table, to cover the overhead of memory
+ * synchronization that makes the hash table slightly slower to build than
+ * a private hash table. There is no extra charge for probing the hash
+ * table for outer path row, on the basis that read-only access to the
+ * hash table shouldn't generate any extra memory synchronization.
+ *
+ * TODO: Really what we want is some guess at the number of cache sync
+ * overhead generated by inserting into cachelines that have been
+ * invalidated by someone else inserting into a bucket in the same
+ * cacheline. Not sure if it's better to introduce a
+ * cpu_cacheline_sync_cost (or _miss_cost?) and then here estimate the
+ * number of collisions we expect based by num buckets, cacheline size,
+ * num workers. But that might be too detailed/low level/variable
+ * heavy/bogus.
+ */
+ if (table_type != HASHPATH_TABLE_PRIVATE)
+ startup_cost += cpu_shared_tuple_cost * inner_path_rows;
+
+ /*
* Get hash table size that executor would use for inner relation.
*
* XXX for the moment, always assume that skew optimization will be
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index cc7384f..87c4cef 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -483,7 +483,8 @@ try_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *hashclauses,
JoinType jointype,
- JoinPathExtraData *extra)
+ JoinPathExtraData *extra,
+ HashPathTableType table_type)
{
Relids required_outer;
JoinCostWorkspace workspace;
@@ -508,7 +509,7 @@ try_hashjoin_path(PlannerInfo *root,
*/
initial_cost_hashjoin(root, &workspace, jointype, hashclauses,
outer_path, inner_path,
- extra->sjinfo, &extra->semifactors);
+ extra->sjinfo, &extra->semifactors, table_type);
if (add_path_precheck(joinrel,
workspace.startup_cost, workspace.total_cost,
@@ -525,7 +526,8 @@ try_hashjoin_path(PlannerInfo *root,
inner_path,
extra->restrictlist,
required_outer,
- hashclauses));
+ hashclauses,
+ table_type));
}
else
{
@@ -546,7 +548,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *hashclauses,
JoinType jointype,
- JoinPathExtraData *extra)
+ JoinPathExtraData *extra,
+ HashPathTableType table_type)
{
JoinCostWorkspace workspace;
@@ -571,7 +574,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
*/
initial_cost_hashjoin(root, &workspace, jointype, hashclauses,
outer_path, inner_path,
- extra->sjinfo, &extra->semifactors);
+ extra->sjinfo, &extra->semifactors,
+ table_type);
if (!add_partial_path_precheck(joinrel, workspace.total_cost, NIL))
return;
@@ -587,7 +591,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
inner_path,
extra->restrictlist,
NULL,
- hashclauses));
+ hashclauses,
+ table_type));
}
/*
@@ -1356,7 +1361,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
/* no possibility of cheap startup here */
}
else if (jointype == JOIN_UNIQUE_INNER)
@@ -1372,7 +1378,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
if (cheapest_startup_outer != NULL &&
cheapest_startup_outer != cheapest_total_outer)
try_hashjoin_path(root,
@@ -1381,7 +1388,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
}
else
{
@@ -1402,7 +1410,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
foreach(lc1, outerrel->cheapest_parameterized_paths)
{
@@ -1436,7 +1445,8 @@ hash_inner_and_outer(PlannerInfo *root,
innerpath,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
}
}
}
@@ -1445,23 +1455,32 @@ hash_inner_and_outer(PlannerInfo *root,
* If the joinrel is parallel-safe, we may be able to consider a
* partial hash join. However, we can't handle JOIN_UNIQUE_OUTER,
* because the outer path will be partial, and therefore we won't be
- * able to properly guarantee uniqueness. Similarly, we can't handle
- * JOIN_FULL and JOIN_RIGHT, because they can produce false null
- * extended rows. Also, the resulting path must not be parameterized.
+ * able to properly guarantee uniqueness. Also, the resulting path
+ * must not be parameterized.
*/
if (joinrel->consider_parallel &&
jointype != JOIN_UNIQUE_OUTER &&
- jointype != JOIN_FULL &&
- jointype != JOIN_RIGHT &&
outerrel->partial_pathlist != NIL &&
bms_is_empty(joinrel->lateral_relids))
{
Path *cheapest_partial_outer;
+ 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? */
+ if (innerrel->partial_pathlist != NIL)
+ cheapest_partial_inner =
+ (Path *) linitial(innerrel->partial_pathlist);
+ if (cheapest_partial_inner != NULL)
+ try_partial_hashjoin_path(root, joinrel,
+ cheapest_partial_outer,
+ cheapest_partial_inner,
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_SHARED_PARALLEL);
+
/*
* Normally, given that the joinrel is parallel-safe, the cheapest
* total inner path will also be parallel-safe, but if not, we'll
@@ -1488,10 +1507,20 @@ hash_inner_and_outer(PlannerInfo *root,
}
if (cheapest_safe_inner != NULL)
+ {
+ /* Try a shared table with only one worker building the table. */
try_partial_hashjoin_path(root, joinrel,
cheapest_partial_outer,
cheapest_safe_inner,
- hashclauses, jointype, extra);
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_SHARED_SERIAL);
+ /* Also private hash tables, built by each worker. */
+ try_partial_hashjoin_path(root, joinrel,
+ cheapest_partial_outer,
+ cheapest_safe_inner,
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_PRIVATE);
+ }
}
}
}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index ad49674..4954c4c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -3938,6 +3938,23 @@ create_hashjoin_plan(PlannerInfo *root,
copy_plan_costsize(&hash_plan->plan, inner_plan);
hash_plan->plan.startup_cost = hash_plan->plan.total_cost;
+ /*
+ * Set the table as sharable if appropriate, with parallel or serial
+ * building.
+ */
+ switch (best_path->table_type)
+ {
+ case HASHPATH_TABLE_SHARED_PARALLEL:
+ hash_plan->shared_table = true;
+ hash_plan->plan.parallel_aware = true;
+ break;
+ case HASHPATH_TABLE_SHARED_SERIAL:
+ hash_plan->shared_table = true;
+ break;
+ case HASHPATH_TABLE_PRIVATE:
+ break;
+ }
+
join_plan = make_hashjoin(tlist,
joinclauses,
otherclauses,
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..68cabe6 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -2096,6 +2096,7 @@ create_mergejoin_path(PlannerInfo *root,
* 'required_outer' is the set of required outer rels
* 'hashclauses' are the RestrictInfo nodes to use as hash clauses
* (this should be a subset of the restrict_clauses list)
+ * 'table_type' for level of hash table sharing
*/
HashPath *
create_hashjoin_path(PlannerInfo *root,
@@ -2108,7 +2109,8 @@ create_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *restrict_clauses,
Relids required_outer,
- List *hashclauses)
+ List *hashclauses,
+ HashPathTableType table_type)
{
HashPath *pathnode = makeNode(HashPath);
@@ -2123,9 +2125,13 @@ create_hashjoin_path(PlannerInfo *root,
sjinfo,
required_outer,
&restrict_clauses);
- pathnode->jpath.path.parallel_aware = false;
+ pathnode->jpath.path.parallel_aware =
+ joinrel->consider_parallel &&
+ (table_type == HASHPATH_TABLE_SHARED_SERIAL ||
+ table_type == HASHPATH_TABLE_SHARED_PARALLEL);
pathnode->jpath.path.parallel_safe = joinrel->consider_parallel &&
outer_path->parallel_safe && inner_path->parallel_safe;
+ pathnode->table_type = table_type;
/* This is a foolish way to estimate parallel_workers, but for now... */
pathnode->jpath.path.parallel_workers = outer_path->parallel_workers;
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index a392197..00619e4 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3393,6 +3393,54 @@ pgstat_get_wait_ipc(WaitEventIPC w)
case WAIT_EVENT_SYNC_REP:
event_name = "SyncRep";
break;
+ case WAIT_EVENT_HASH_CREATING:
+ event_name = "Hash/Creating";
+ break;
+ case WAIT_EVENT_HASH_HASHING:
+ event_name = "Hash/Hashing";
+ break;
+ case WAIT_EVENT_HASH_RESIZING:
+ event_name = "Hash/Resizing";
+ break;
+ case WAIT_EVENT_HASH_REBUCKETING:
+ event_name = "Hash/Rebucketing";
+ break;
+ case WAIT_EVENT_HASH_INIT:
+ event_name = "Hash/Init";
+ break;
+ case WAIT_EVENT_HASH_DESTROY:
+ event_name = "Hash/Destroy";
+ break;
+ case WAIT_EVENT_HASH_UNMATCHED:
+ event_name = "Hash/Unmatched";
+ break;
+ case WAIT_EVENT_HASH_PROMOTING:
+ event_name = "Hash/Promoting";
+ break;
+ case WAIT_EVENT_HASHJOIN_PROMOTING:
+ event_name = "HashJoin/Promoting";
+ break;
+ case WAIT_EVENT_HASHJOIN_PROBING:
+ event_name = "HashJoin/Probing";
+ break;
+ case WAIT_EVENT_HASHJOIN_SKIP_LOADING:
+ event_name = "HashJoin/SkipLoading";
+ break;
+ case WAIT_EVENT_HASHJOIN_SKIP_PROBING:
+ event_name = "HashJoin/SkipProbing";;
+ break;
+ case WAIT_EVENT_HASHJOIN_LOADING:
+ event_name = "HashJoin/Loading";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING:
+ event_name = "HashJoin/Rewinding";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING2:
+ event_name = "HashJoin/Rewinding2";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING3:
+ event_name = "HashJoin/Rewinding3";;
+ break;
/* no default case, so that compiler will warn */
}
diff --git a/src/backend/storage/file/buffile.c b/src/backend/storage/file/buffile.c
index 042be79..b38cbd8 100644
--- a/src/backend/storage/file/buffile.c
+++ b/src/backend/storage/file/buffile.c
@@ -42,6 +42,8 @@
#include "storage/buf_internals.h"
#include "utils/resowner.h"
+extern int ParallelWorkerNumber;
+
/*
* We break BufFiles into gigabyte-sized segments, regardless of RELSEG_SIZE.
* The reason is that we'd like large temporary BufFiles to be spread across
@@ -89,6 +91,24 @@ struct BufFile
char buffer[BLCKSZ];
};
+/*
+ * Serialized representation of a single file managed by a BufFile.
+ */
+typedef struct BufFileFileDescriptor
+{
+ char path[MAXPGPATH];
+} BufFileFileDescriptor;
+
+/*
+ * Serialized representation of a BufFile, to be created by BufFileExport and
+ * consumed by BufFileImport.
+ */
+struct BufFileDescriptor
+{
+ size_t num_files;
+ BufFileFileDescriptor files[FLEXIBLE_ARRAY_MEMBER];
+};
+
static BufFile *makeBufFile(File firstfile);
static void extendBufFile(BufFile *file);
static void BufFileLoadBuffer(BufFile *file);
@@ -178,6 +198,83 @@ BufFileCreateTemp(bool interXact)
return file;
}
+/*
+ * Export a BufFile description in a serialized form so that another backend
+ * can attach to it and read from it. The format is opaque, but it may be
+ * bitwise copied, and its size may be obtained with BufFileDescriptorSize().
+ */
+BufFileDescriptor *
+BufFileExport(BufFile *file)
+{
+ BufFileDescriptor *descriptor;
+ int i;
+
+ /* Flush output from local buffers. */
+ BufFileFlush(file);
+
+ /*
+ * TODO: FIXME: disable cleanup until I can figure out a decent cleanup
+ * strategy!
+ */
+ file->isInterXact = true;
+
+ /* Create and fill in a descriptor. */
+ descriptor = palloc0(offsetof(BufFileDescriptor, files) +
+ sizeof(BufFileFileDescriptor) * file->numFiles);
+ descriptor->num_files = file->numFiles;
+ for (i = 0; i < descriptor->num_files; ++i)
+ strcpy(descriptor->files[i].path, FilePathName(file->files[i]));
+
+ return descriptor;
+}
+
+/*
+ * Return the size in bytes of a BufFileDescriptor, so that it can be copied.
+ */
+size_t
+BufFileDescriptorSize(const BufFileDescriptor *descriptor)
+{
+ return offsetof(BufFileDescriptor, files) +
+ sizeof(BufFileFileDescriptor) * descriptor->num_files;
+}
+
+/*
+ * Open a BufFile that was created by another backend and then exported. The
+ * file must be read-only in all backends, and is still owned by the backend
+ * that created it. This provides a way for cooperating backends to share
+ * immutable temporary data such as hash join batches.
+ */
+BufFile *
+BufFileImport(BufFileDescriptor *descriptor)
+{
+ BufFile *file = (BufFile *) palloc(sizeof(BufFile));
+ int i;
+
+ file->numFiles = descriptor->num_files;
+ file->files = (File *) palloc0(sizeof(File) * descriptor->num_files);
+ file->offsets = (off_t *) palloc0(sizeof(off_t) * descriptor->num_files);
+ file->isTemp = false;
+ file->isInterXact = true; /* prevent cleanup by this backend */
+ file->dirty = false;
+ file->resowner = CurrentResourceOwner;
+ file->curFile = 0;
+ file->curOffset = 0L;
+ file->pos = 0;
+ file->nbytes = 0;
+
+ for (i = 0; i < descriptor->num_files; ++i)
+ {
+ file->files[i] =
+ PathNameOpenFile(descriptor->files[i].path,
+ O_RDONLY | PG_BINARY, 0600);
+ if (file->files[i] <= 0)
+ elog(ERROR, "failed to import \"%s\": %m",
+ descriptor->files[i].path);
+ }
+
+ return file;
+}
+
#ifdef NOT_USED
/*
* Create a BufFile and attach it to an already-opened virtual File.
diff --git a/src/backend/storage/ipc/barrier.c b/src/backend/storage/ipc/barrier.c
index 8b83c1d..5a45103 100644
--- a/src/backend/storage/ipc/barrier.c
+++ b/src/backend/storage/ipc/barrier.c
@@ -16,6 +16,7 @@
#include "storage/barrier.h"
+
/*
* Initialize this barrier, setting a static number of participants that we
* will wait for at each computation phase. To use a dynamic number of
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 2d3cf9e..9becab0 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -749,6 +749,7 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
}
/* Values only available to role member */
+ elog(LOG, "XXX pid %d -> %d", beentry->st_procpid, has_privs_of_role(GetUserId(), beentry->st_userid));
if (has_privs_of_role(GetUserId(), beentry->st_userid))
{
SockAddr zero_clientaddr;
@@ -788,7 +789,6 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
raw_wait_event = UINT32_ACCESS_ONCE(proc->wait_event_info);
wait_event_type = pgstat_get_wait_event_type(raw_wait_event);
wait_event = pgstat_get_wait_event(raw_wait_event);
-
}
else
{
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 65660c1..9b49918 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2857,6 +2857,16 @@ static struct config_real ConfigureNamesReal[] =
NULL, NULL, NULL
},
{
+ {"cpu_shared_tuple_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "sharing each tuple with other parallel workers."),
+ NULL
+ },
+ &cpu_shared_tuple_cost,
+ DEFAULT_CPU_TUPLE_COST, -DBL_MAX, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
{"cpu_index_tuple_cost", PGC_USERSET, QUERY_TUNING_COST,
gettext_noop("Sets the planner's estimate of the cost of "
"processing each index entry during an index scan."),
diff --git a/src/include/executor/hashjoin.h b/src/include/executor/hashjoin.h
index 6d0e12b..715d420 100644
--- a/src/include/executor/hashjoin.h
+++ b/src/include/executor/hashjoin.h
@@ -15,7 +15,13 @@
#define HASHJOIN_H
#include "nodes/execnodes.h"
+#include "port/atomics.h"
+#include "storage/barrier.h"
#include "storage/buffile.h"
+#include "storage/dsa.h"
+#include "storage/fd.h"
+#include "storage/lwlock.h"
+#include "storage/spin.h"
/* ----------------------------------------------------------------
* hash-join hash table structures
@@ -63,7 +69,12 @@
typedef struct HashJoinTupleData
{
- struct HashJoinTupleData *next; /* link to next tuple in same bucket */
+ /* link to next tuple in same bucket */
+ union
+ {
+ dsa_pointer shared;
+ struct HashJoinTupleData *private;
+ } next;
uint32 hashvalue; /* tuple's hash code */
/* Tuple data, in MinimalTuple format, follows on a MAXALIGN boundary */
} HashJoinTupleData;
@@ -94,7 +105,12 @@ typedef struct HashJoinTupleData
typedef struct HashSkewBucket
{
uint32 hashvalue; /* common hash value */
- HashJoinTuple tuples; /* linked list of inner-relation tuples */
+ /* linked list of inner-relation tuples */
+ union
+ {
+ dsa_pointer shared;
+ HashJoinTuple private;
+ } tuples;
} HashSkewBucket;
#define SKEW_BUCKET_OVERHEAD MAXALIGN(sizeof(HashSkewBucket))
@@ -103,8 +119,9 @@ typedef struct HashSkewBucket
#define SKEW_MIN_OUTER_FRACTION 0.01
/*
- * To reduce palloc overhead, the HashJoinTuples for the current batch are
- * packed in 32kB buffers instead of pallocing each tuple individually.
+ * To reduce palloc/dsa_allocate overhead, the HashJoinTuples for the current
+ * batch are packed in 32kB buffers instead of pallocing each tuple
+ * individually.
*/
typedef struct HashMemoryChunkData
{
@@ -112,17 +129,118 @@ 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
+ {
+ dsa_pointer shared;
+ struct HashMemoryChunkData *private;
+ } 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_THRESHOLD (HASH_CHUNK_SIZE / 4)
+/*
+ * Read head position in a shared batch file.
+ */
+typedef struct HashJoinBatchPosition
+{
+ int fileno;
+ off_t offset;
+} HashJoinBatchPosition;
+
+/*
+ * The state exposed in shared memory for each participant to coordinate
+ * reading of batch files that it wrote.
+ */
+typedef struct HashJoinSharedBatchReader
+{
+ int batchno; /* the batch number we are currently reading */
+
+ LWLock lock; /* protects access to the members below */
+ bool error; /* has an IO error occurred? */
+ HashJoinBatchPosition head; /* shared read head for current batch */
+} HashJoinSharedBatchReader;
+
+/*
+ * The state exposed in shared memory by each participant allowing its batch
+ * files to be read by other participants.
+ */
+typedef struct HashJoinParticipantState
+{
+ /*
+ * Arrays of pointers to arrays of pointers to BufFileDesciptor objects
+ * exported by this participant. The descriptor for batch i is in slot
+ * i % (1 << fls(i - 1)) of the array at index fls(i).
+ *
+ * This arrangement means that we can modify future batches without
+ * moving/reallocating the current batch. The current batch is therefore
+ * immutable and accessible by other backends which need to read it.
+ */
+ dsa_pointer inner_batch_descriptors[32]; /* number of bits in batchno */
+ dsa_pointer outer_batch_descriptors[32];
+
+ /*
+ * The shared state used to coordinate reading from the current batch. We
+ * need separate objects for the outer and inner side, because in the
+ * probing phase some participants can be reading from the outer batch,
+ * while others can be reading from the inner side to preload the next
+ * batch.
+ */
+ HashJoinSharedBatchReader inner_batch_reader;
+ HashJoinSharedBatchReader outer_batch_reader;
+} HashJoinParticipantState;
+
+/*
+ * The state used by each backend to manage reading from batch files written
+ * by all participants.
+ */
+typedef struct HashJoinBatchReader
+{
+ int participant_number; /* read which participant's batch? */
+ HashJoinSharedBatchReader *shared; /* holder of the shared read head */
+ BufFile *file; /* the file opened in this backend */
+ HashJoinBatchPosition head; /* local read head position */
+} HashJoinBatchReader;
+
+/*
+ * State for a shared hash join table. Each backend participating in a hash
+ * join with a shared hash table also has a HashJoinTableData object in
+ * backend-private memory, which points to this shared state in the DSM
+ * segment.
+ */
+typedef struct SharedHashJoinTableData
+{
+ Barrier barrier; /* for synchronizing workers */
+ dsa_pointer primary_buckets; /* primary hash table */
+ dsa_pointer secondary_buckets; /* hash table for preloading next batch */
+ bool at_least_one_worker; /* did at least one worker join in time? */
+ int nbuckets;
+ int nbuckets_optimal;
+ pg_atomic_uint32 next_unmatched_bucket;
+ pg_atomic_uint64 total_primary_tuples;
+ pg_atomic_uint64 total_secondary_tuples;
+ dsa_pointer_atomic head_primary_chunk;
+ dsa_pointer_atomic head_secondary_chunk;
+ dsa_pointer_atomic chunks_to_rebucket;
+ int planned_participants; /* number of planned workers + leader */
+
+ /* state exposed by each participant for sharing batches */
+ HashJoinParticipantState participants[FLEXIBLE_ARRAY_MEMBER];
+} SharedHashJoinTableData;
+
+typedef union HashJoinBucketHead
+{
+ dsa_pointer_atomic shared;
+ HashJoinTuple private;
+} HashJoinBucketHead;
+
typedef struct HashJoinTableData
{
int nbuckets; /* # buckets in the in-memory hash table */
@@ -134,9 +252,11 @@ typedef struct HashJoinTableData
int log2_nbuckets_optimal; /* log2(nbuckets_optimal) */
/* buckets[i] is head of list of tuples in i'th in-memory bucket */
- struct HashJoinTupleData **buckets;
+ HashJoinBucketHead *buckets;
/* buckets array is per-batch storage, as are all the tuples */
+ HashJoinBucketHead *next_buckets; /* for preloading next batch */
+
bool keepNulls; /* true to store unmatchable NULL tuples */
bool skewEnabled; /* are we using skew optimization? */
@@ -185,7 +305,71 @@ typedef struct HashJoinTableData
MemoryContext batchCxt; /* context for this-batch-only storage */
/* used for dense allocation of tuples (into linked chunks) */
- HashMemoryChunk chunks; /* one list for the whole batch */
+ HashMemoryChunk primary_chunk; /* current chunk for this batch */
+ HashMemoryChunk secondary_chunk; /* current chunk for next batch */
+ HashMemoryChunk chunks_to_rebucket; /* after resizing table */
+ dsa_pointer primary_chunk_shared; /* DSA pointer to primary_chunk */
+ dsa_pointer secondary_chunk_shared; /* DSA pointer to secondary_chunk */
+
+ /* State for coordinating shared tables for parallel hash joins. */
+ dsa_area *area;
+ SharedHashJoinTableData *shared; /* the shared state */
+ int attached_at_phase; /* the phase this participant joined */
+ bool detached_early; /* did we decide to detach early? */
+ HashJoinBatchReader batch_reader; /* state for reading batches in */
} HashJoinTableData;
+/* Check if a HashJoinTable is shared by parallel workers. */
+#define HashJoinTableIsShared(table) ((table)->shared != NULL)
+
+/* The phases of parallel hash computation. */
+#define PHJ_PHASE_INIT 0
+#define PHJ_PHASE_CREATING 1
+#define PHJ_PHASE_HASHING 2
+#define PHJ_PHASE_RESIZING 3
+#define PHJ_PHASE_REBUCKETING 4
+#define PHJ_PHASE_PROBING 5 /* PHJ_PHASE_PROBING_BATCH(0) */
+#define PHJ_PHASE_UNMATCHED 6 /* PHJ_PHASE_UNMATCHED_BATCH(0) */
+
+/* The subphases for batches. */
+#define PHJ_SUBPHASE_PROMOTING 0
+#define PHJ_SUBPHASE_LOADING 1
+#define PHJ_SUBPHASE_PROBING 2
+#define PHJ_SUBPHASE_UNMATCHED 3
+
+/* The phases of parallel processing for batch(n). */
+#define PHJ_PHASE_PROMOTING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 4 - 3)
+#define PHJ_PHASE_LOADING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 4 - 2)
+#define PHJ_PHASE_PROBING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 4 - 1)
+#define PHJ_PHASE_UNMATCHED_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 4 - 0)
+
+/* Phase number -> sub-phase within a batch. */
+#define PHJ_PHASE_TO_SUBPHASE(p) \
+ (((int)(p) - PHJ_PHASE_UNMATCHED + PHJ_SUBPHASE_UNMATCHED) % 4)
+
+/* Phase number -> batch number. */
+#define PHJ_PHASE_TO_BATCHNO(p) \
+ (((int)(p) - PHJ_PHASE_UNMATCHED + PHJ_SUBPHASE_UNMATCHED) / 4)
+
+/*
+ * Is a given phase one in which a new hash table array is being assigned by
+ * one elected backend? That includes initial creation, reallocation during
+ * resize, and promotion of secondary hash table to primary. Workers that
+ * show up and attach at an arbitrary time must wait such phases out before
+ * doing anything with the hash table.
+ */
+#define PHJ_PHASE_MUTATING_TABLE(p) \
+ ((p) == PHJ_PHASE_CREATING || \
+ (p) == PHJ_PHASE_RESIZING || \
+ (PHJ_PHASE_TO_BATCHNO(p) > 0 && \
+ PHJ_PHASE_TO_SUBPHASE(p) == PHJ_SUBPHASE_PROMOTING))
+
+/*
+ * Return the 'participant number' for a process participating in a parallel
+ * hash join. We give a number < hashtable->shared->planned_participants
+ * to each potential participant, including the leader.
+ */
+#define HashJoinParticipantNumber() \
+ (IsParallelWorker() ? ParallelWorkerNumber + 1 : 0)
+
#endif /* HASHJOIN_H */
diff --git a/src/include/executor/nodeHash.h b/src/include/executor/nodeHash.h
index 8cf6d15..d208981 100644
--- a/src/include/executor/nodeHash.h
+++ b/src/include/executor/nodeHash.h
@@ -22,12 +22,12 @@ extern Node *MultiExecHash(HashState *node);
extern void ExecEndHash(HashState *node);
extern void ExecReScanHash(HashState *node);
-extern HashJoinTable ExecHashTableCreate(Hash *node, List *hashOperators,
+extern HashJoinTable ExecHashTableCreate(HashState *node, List *hashOperators,
bool keepNulls);
extern void ExecHashTableDestroy(HashJoinTable hashtable);
extern void ExecHashTableInsert(HashJoinTable hashtable,
TupleTableSlot *slot,
- uint32 hashvalue);
+ uint32 hashvalue, bool secondary);
extern bool ExecHashGetHashValue(HashJoinTable hashtable,
ExprContext *econtext,
List *hashkeys,
@@ -49,5 +49,8 @@ extern void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
int *numbatches,
int *num_skew_mcvs);
extern int ExecHashGetSkewBucket(HashJoinTable hashtable, uint32 hashvalue);
+extern void ExecHashPreloadNextBatch(HashJoinTable hashtable);
+extern void ExecHashUpdate(HashJoinTable hashtable);
+extern bool ExecHashCheckForEarlyExit(HashJoinTable hashtable);
#endif /* NODEHASH_H */
diff --git a/src/include/executor/nodeHashjoin.h b/src/include/executor/nodeHashjoin.h
index f24127a..7d07788 100644
--- a/src/include/executor/nodeHashjoin.h
+++ b/src/include/executor/nodeHashjoin.h
@@ -14,15 +14,25 @@
#ifndef NODEHASHJOIN_H
#define NODEHASHJOIN_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
#include "storage/buffile.h"
+#include "storage/shm_toc.h"
extern HashJoinState *ExecInitHashJoin(HashJoin *node, EState *estate, int eflags);
extern TupleTableSlot *ExecHashJoin(HashJoinState *node);
extern void ExecEndHashJoin(HashJoinState *node);
extern void ExecReScanHashJoin(HashJoinState *node);
-extern void ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
- BufFile **fileptr);
+extern void ExecHashJoinSaveTuple(HashJoinTable hashtable,
+ MinimalTuple tuple, uint32 hashvalue,
+ int batchno, bool inner);
+extern void ExecHashJoinInitializeBatchReader(HashJoinTable hashtable,
+ int batchno, bool inner);
+extern void ExecHashJoinResetBatchReaders(HashJoinTable hashtable);
+
+extern void ExecHashJoinEstimate(HashJoinState *state, ParallelContext *pcxt);
+extern void ExecHashJoinInitializeDSM(HashJoinState *state, ParallelContext *pcxt);
+extern void ExecHashJoinInitializeWorker(HashJoinState *state, shm_toc *toc);
#endif /* NODEHASHJOIN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 2fadf76..9ae55be 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1738,6 +1738,7 @@ typedef struct MergeJoinState
/* these structs are defined in executor/hashjoin.h: */
typedef struct HashJoinTupleData *HashJoinTuple;
typedef struct HashJoinTableData *HashJoinTable;
+typedef struct SharedHashJoinTableData *SharedHashJoinTable;
typedef struct HashJoinState
{
@@ -1759,6 +1760,7 @@ typedef struct HashJoinState
int hj_JoinState;
bool hj_MatchedOuter;
bool hj_OuterNotEmpty;
+ SharedHashJoinTable hj_sharedHashJoinTable;
} HashJoinState;
@@ -1982,6 +1984,9 @@ typedef struct HashState
HashJoinTable hashtable; /* hash table for the hashjoin */
List *hashkeys; /* list of ExprState nodes */
/* hashkeys is same as parent's hj_InnerHashKeys */
+
+ /* The following are the same as the parent's. */
+ SharedHashJoinTable shared_table_data;
} HashState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index e2fbc7d..e8f90d9 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -782,6 +782,7 @@ typedef struct Hash
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 */
+ bool shared_table; /* table shared by multiple workers? */
/* all other info is in the parent HashJoin node */
} Hash;
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..8b06551 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1258,6 +1258,16 @@ typedef struct MergePath
bool materialize_inner; /* add Materialize to inner? */
} MergePath;
+typedef enum
+{
+ /* Every worker builds its own private copy of the hash table. */
+ HASHPATH_TABLE_PRIVATE,
+ /* One worker builds a shared hash table, and all workers probe it. */
+ HASHPATH_TABLE_SHARED_SERIAL,
+ /* All workers build a shared hash table, and then probe it. */
+ HASHPATH_TABLE_SHARED_PARALLEL
+} HashPathTableType;
+
/*
* A hashjoin path has these fields.
*
@@ -1272,6 +1282,7 @@ typedef struct HashPath
JoinPath jpath;
List *path_hashclauses; /* join clauses used for hashing */
int num_batches; /* number of batches expected */
+ HashPathTableType table_type; /* level of sharedness */
} HashPath;
/*
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..7bb0d1d 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -24,6 +24,7 @@
#define DEFAULT_SEQ_PAGE_COST 1.0
#define DEFAULT_RANDOM_PAGE_COST 4.0
#define DEFAULT_CPU_TUPLE_COST 0.01
+#define DEFAULT_CPU_SHARED_TUPLE_COST 0.0
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
#define DEFAULT_PARALLEL_TUPLE_COST 0.1
@@ -48,6 +49,7 @@ typedef enum
extern PGDLLIMPORT double seq_page_cost;
extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
+extern PGDLLIMPORT double cpu_shared_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
extern PGDLLIMPORT double parallel_tuple_cost;
@@ -144,7 +146,8 @@ extern void initial_cost_hashjoin(PlannerInfo *root,
List *hashclauses,
Path *outer_path, Path *inner_path,
SpecialJoinInfo *sjinfo,
- SemiAntiJoinFactors *semifactors);
+ SemiAntiJoinFactors *semifactors,
+ HashPathTableType table_type);
extern void final_cost_hashjoin(PlannerInfo *root, HashPath *path,
JoinCostWorkspace *workspace,
SpecialJoinInfo *sjinfo,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..5f4ca87 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -134,7 +134,8 @@ extern HashPath *create_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *restrict_clauses,
Relids required_outer,
- List *hashclauses);
+ List *hashclauses,
+ HashPathTableType table_type);
extern ProjectionPath *create_projection_path(PlannerInfo *root,
RelOptInfo *rel,
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 0b85b7a..519b2e6 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -785,7 +785,23 @@ typedef enum
WAIT_EVENT_MQ_SEND,
WAIT_EVENT_PARALLEL_FINISH,
WAIT_EVENT_SAFE_SNAPSHOT,
- WAIT_EVENT_SYNC_REP
+ WAIT_EVENT_SYNC_REP,
+ WAIT_EVENT_HASH_CREATING,
+ WAIT_EVENT_HASH_HASHING,
+ WAIT_EVENT_HASH_RESIZING,
+ WAIT_EVENT_HASH_REBUCKETING,
+ WAIT_EVENT_HASH_INIT,
+ WAIT_EVENT_HASH_DESTROY,
+ WAIT_EVENT_HASH_UNMATCHED,
+ WAIT_EVENT_HASH_PROMOTING,
+ WAIT_EVENT_HASHJOIN_PROMOTING,
+ WAIT_EVENT_HASHJOIN_PROBING,
+ WAIT_EVENT_HASHJOIN_SKIP_LOADING,
+ WAIT_EVENT_HASHJOIN_SKIP_PROBING,
+ WAIT_EVENT_HASHJOIN_LOADING,
+ WAIT_EVENT_HASHJOIN_REWINDING,
+ WAIT_EVENT_HASHJOIN_REWINDING2, /* TODO: rename me */
+ WAIT_EVENT_HASHJOIN_REWINDING3 /* TODO: rename me */
} WaitEventIPC;
/* ----------
diff --git a/src/include/storage/buffile.h b/src/include/storage/buffile.h
index 809e596..044262d 100644
--- a/src/include/storage/buffile.h
+++ b/src/include/storage/buffile.h
@@ -30,12 +30,17 @@
typedef struct BufFile BufFile;
+typedef struct BufFileDescriptor BufFileDescriptor;
+
/*
* prototypes for functions in buffile.c
*/
extern BufFile *BufFileCreateTemp(bool interXact);
extern void BufFileClose(BufFile *file);
+extern BufFileDescriptor *BufFileExport(BufFile *file);
+extern BufFile *BufFileImport(BufFileDescriptor *descriptor);
+extern size_t BufFileDescriptorSize(const BufFileDescriptor *descriptor);
extern size_t BufFileRead(BufFile *file, void *ptr, size_t size);
extern size_t BufFileWrite(BufFile *file, void *ptr, size_t size);
extern int BufFileSeek(BufFile *file, int fileno, off_t offset, int whence);
On Tue, Nov 1, 2016 at 5:33 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
Please find a WIP patch attached. Everything related to batch reading
is not currently in a working state, which breaks multi-batch joins,
but many single batch cases work correctly. In an earlier version I
had multi-batch joins working but was before I started tackling
problems 2 and 3 listed in my earlier message.
Here is a better version with code to handle multi-batch joins. The
BufFile sharing approach to reading other participants' batch files is
a straw-man (perhaps what we really want would look more like a shared
tuplestore?), but solves the immediate problem I described earlier so
I can focus on other aspects of the problem. There may be some issues
with cleanup though, more on that soon.
Here's a summary of how this patch chops the hash join up into phases.
The 'phase' is an integer that encodes the step we're up to in the
algorithm, including the current batch number, and I represent that
with macros like PHJ_PHASE_HASHING and PHJ_PHASE_PROBING_BATCH(42).
Each phase is either serial, meaning that one participant does
something special, or parallel meaning that all participants do the
same thing. It goes like this:
* PHJ_PHASE_INIT
The initial phase established by the leader before launching workers.
* PHJ_PHASE_CREATING
Serial: One participant creates the hash table.
* PHJ_PHASE_HASHING
Serial or parallel: Depending on plan, one or all participants
execute the inner plan to completion, building the hash table for
batch 0 and possibly writing tuples to batch files on disk for future
batches.
* PHJ_PHASE_RESIZING
Serial: One participant resizes the hash table if necessary.
* PHJ_PHASE_REBUCKETING
Parallel: If the hash table was resized, all participants rehash all
the tuples in it and insert them into the buckets of the new larger
hash table.
* PHJ_PHASE_PROBING_BATCH(0)
Parallel: All participants execute the outer plan to completion. For
each tuple they either probe the hash table if it's for the current
batch, or write it out to a batch file if it's for a future batch.
For each tuple matched in the hash table, they set the matched bit.
When they are finished probing batch 0, they also preload tuples from
inner batch 1 into a secondary hash table until work_mem is exhausted
(note that at this time work_mem is occupied by the primary hash
table: this is just a way to use any remaining work_mem and extract a
little bit more parallelism, since otherwise every participant would
be waiting for all participants to finish probing; instead we wait for
all paticipants to finish probing AND for spare work_mem to run out).
* PHJ_PHASE_UNMATCHED_BATCH(0)
Parallel: For right/full joins, all participants then scan the hash
table looking for unmatched tuples.
... now we are ready for batch 1 ...
* PHJ_PHASE_PROMOTING_BATCH(1)
Serial: One participant promotes the secondary hash table to become
the new primary hash table.
* PHJ_PHASE_LOADING_BATCH(1)
Parallel: All participants finish loading inner batch 1 into the hash
table (work that was started in the previous probing phase).
* PHJ_PHASE_PREPARING_BATCH(1)
Serial: One participant resets the batch reading heads, so that we
are ready to read from outer batch 1, and inner batch 2.
* PHJ_PHASE_PROBING_BATCH(1)
Parallel: All participants read from outer batch 1 to probe the hash
table, then read from inner batch 2 to preload tuples into the
secondary hash table.
* PHJ_PHASE_UNMATCHED_BATCH(1)
Parallel: For right/full joins, all participants then scan the hash
table looking for unmatched tuples.
... now we are ready for batch 2 ...
Then all participants synchronise a final time to enter batch
PHJ_PHASE_PROMOTING_BATCH(nbatch), which is one past the end and is
the point at which it is safe to clean up. (There may be an
optimisation where I can clean up after the last participant detaches
instead, more on that soon).
Obviously I'm actively working on developing and stabilising all this.
Some of the things I'm working on are: work_mem accounting, batch
increases, rescans and figuring out if the resource management for
those BufFiles is going to work. There are quite a lot of edge cases
some of which I'm still figuring out, but I feel like this approach is
workable. At this stage I want to share what I'm doing to see if
others have feedback, ideas, blood curdling screams of horror, etc. I
will have better patches and a set of test queries soon. Thanks for
reading.
--
Thomas Munro
http://www.enterprisedb.com
Attachments:
parallel-hash-v2.patchapplication/octet-stream; name=parallel-hash-v2.patchDownload
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 0a669d9..1e7d369 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -1023,7 +1023,10 @@ ExplainNode(PlanState *planstate, List *ancestors,
pname = sname = "Limit";
break;
case T_Hash:
- pname = sname = "Hash";
+ if (((Hash *) plan)->shared_table)
+ pname = sname = "Shared Hash";
+ else
+ pname = sname = "Hash";
break;
default:
pname = sname = "???";
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 72bacd5..c8c39f7 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -27,6 +27,7 @@
#include "executor/executor.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
+#include "executor/nodeHashjoin.h"
#include "executor/nodeSeqscan.h"
#include "executor/tqueue.h"
#include "nodes/nodeFuncs.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
break;
+ case T_HashJoinState:
+ ExecHashJoinEstimate((HashJoinState *) planstate,
+ e->pcxt);
+ break;
default:
break;
}
@@ -255,6 +260,9 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
break;
+ case T_HashJoinState:
+ ExecHashJoinInitializeDSM((HashJoinState *) planstate,
+ d->pcxt);
default:
break;
}
@@ -724,6 +732,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
break;
+ case T_HashJoinState:
+ ExecHashJoinInitializeWorker((HashJoinState *) planstate,
+ toc);
+ break;
default:
break;
}
diff --git a/src/backend/executor/nodeHash.c b/src/backend/executor/nodeHash.c
index 6375d9b..0b8d27b 100644
--- a/src/backend/executor/nodeHash.c
+++ b/src/backend/executor/nodeHash.c
@@ -25,6 +25,7 @@
#include <limits.h>
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "catalog/pg_statistic.h"
#include "commands/tablespace.h"
#include "executor/execdebug.h"
@@ -32,12 +33,13 @@
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
#include "miscadmin.h"
+#include "pgstat.h"
+#include "port/atomics.h"
#include "utils/dynahash.h"
#include "utils/memutils.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
-
static void ExecHashIncreaseNumBatches(HashJoinTable hashtable);
static void ExecHashIncreaseNumBuckets(HashJoinTable hashtable);
static void ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node,
@@ -47,8 +49,30 @@ static void ExecHashSkewTableInsert(HashJoinTable hashtable,
uint32 hashvalue,
int bucketNumber);
static void ExecHashRemoveNextSkewBucket(HashJoinTable hashtable);
+static void ExecHashRebucket(HashJoinTable hashtable);
+static void ExecHashTableComputeOptimalBuckets(HashJoinTable hashtable);
+
+static void add_tuple_count(HashJoinTable hashtable, int count,
+ bool secondary);
+static HashJoinTuple next_tuple_in_bucket(HashJoinTable table,
+ HashJoinTuple tuple);
+static HashJoinTuple first_tuple_in_skew_bucket(HashJoinTable table,
+ int skew_bucket_no);
+static HashJoinTuple first_tuple_in_skew_bucket(HashJoinTable table,
+ int bucket_no);
+static void insert_tuple_into_bucket(HashJoinTable table, int bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_pointer);
+static void insert_tuple_into_skew_bucket(HashJoinTable table,
+ int bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_pointer);
static void *dense_alloc(HashJoinTable hashtable, Size size);
+static void *dense_alloc_shared(HashJoinTable hashtable, Size size,
+ dsa_pointer *chunk_shared,
+ bool secondary);
+
/* ----------------------------------------------------------------
* ExecHash
@@ -64,6 +88,100 @@ ExecHash(HashState *node)
}
/* ----------------------------------------------------------------
+ * ExecHashCheckForEarlyExit
+ *
+ * return true if this process needs to abandon work on the
+ * hash join to avoid a deadlock
+ * ----------------------------------------------------------------
+ */
+bool
+ExecHashCheckForEarlyExit(HashJoinTable hashtable)
+{
+ /*
+ * The golden rule of leader deadlock avoidance: since leader processes
+ * have two separate roles, namely reading from worker queues AND executing
+ * the same plan as workers, we must never allow a leader to wait for
+ * workers if there is any possibility those workers have emitted tuples.
+ * Otherwise we could get into a situation where a worker fills up its
+ * output tuple queue and begins waiting for the leader to read, while
+ * the leader is busy waiting for the worker.
+ *
+ * Parallel hash joins with shared tables are inherently susceptible to
+ * such deadlocks because there are points at which all participants must
+ * wait (you can't start check for unmatched tuples in the hash table until
+ * probing has completed in all workers, etc).
+ *
+ * So we follow these rules:
+ *
+ * 1. If there are workers participating, the leader MUST NOT not
+ * participate in any further work after probing the first batch, so
+ * that it never has to wait for workers that might have emitted
+ * tuples.
+ *
+ * 2. If there are no workers participating, the leader MUST run all the
+ * batches to completion, because that's the only way for the join
+ * to complete. There is no deadlock risk if there are no workers.
+ *
+ * 3. Workers MUST NOT participate if the hashing phase has finished by
+ * the time they have joined, so that the leader can reliably determine
+ * whether there are any workers running when it comes to the point
+ * where it must choose between 1 and 2.
+ *
+ * In other words, if the leader makes it all the way through hashing and
+ * probing before any workers show up, then the leader will run the whole
+ * hash join on its own. If workers do show up any time before hashing is
+ * finished, the leader will stop executing the join after helping probe
+ * the first batch. In the unlikely event of the first worker showing up
+ * after the leader has finished hashing, it will exit because it's too
+ * late, the leader has already decided to do all the work alone.
+ */
+
+ if (!IsParallelWorker())
+ {
+ /* Running in the leader process. */
+ if (BarrierPhase(&hashtable->shared->barrier) == PHJ_PHASE_PROBING &&
+ hashtable->shared->at_least_one_worker)
+ {
+ /* Abandon ship due to rule 1. There are workers running. */
+ hashtable->detached_early = true;
+ }
+ else
+ {
+ /*
+ * Continue processing due to rule 2. There are no workers, and
+ * any workers that show up later will abandon ship.
+ */
+ }
+ }
+ else
+ {
+ /* Running in a worker process. */
+ if (hashtable->attached_at_phase < PHJ_PHASE_PROBING)
+ {
+ /*
+ * Advertise that there are workers, so that the leader can
+ * choose between rules 1 and 2. It's OK that several workers can
+ * write to this variable without immediately memory
+ * synchronization, because the leader will only read it in a later
+ * phase (see above).
+ */
+ hashtable->shared->at_least_one_worker = true;
+ }
+ else
+ {
+ /* Abandon ship due to rule 3. */
+ hashtable->detached_early = true;
+ }
+ }
+
+ /* If we decided to exit early, detach now. */
+ if (hashtable->detached_early)
+ BarrierDetach(&hashtable->shared->barrier);
+
+ return hashtable->detached_early;
+}
+
+/* ----------------------------------------------------------------
* MultiExecHash
*
* build hash table for hashjoin, doing partitioning if more
@@ -79,6 +197,7 @@ MultiExecHash(HashState *node)
TupleTableSlot *slot;
ExprContext *econtext;
uint32 hashvalue;
+ Barrier *barrier = NULL;
/* must provide our own instrumentation support */
if (node->ps.instrument)
@@ -90,6 +209,55 @@ MultiExecHash(HashState *node)
outerNode = outerPlanState(node);
hashtable = node->hashtable;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Synchronize parallel hash table builds. At this stage we know that
+ * the shared hash table has been created, but we don't know if our
+ * peers are still in MultiExecHash and if so how far through. We use
+ * the phase to synchronize with them.
+ */
+ barrier = &hashtable->shared->barrier;
+
+ switch (BarrierPhase(barrier))
+ {
+ case PHJ_PHASE_INIT:
+ /* ExecHashTableCreate already handled this phase. */
+ Assert(false);
+ case PHJ_PHASE_CREATING:
+ /* Wait for serial phase, and then either hash or wait. */
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_CREATING))
+ goto hash;
+ else if (node->ps.plan->parallel_aware)
+ goto hash;
+ else
+ goto post_hash;
+ case PHJ_PHASE_HASHING:
+ /* Hashing is already underway. Can we join in? */
+ if (node->ps.plan->parallel_aware)
+ goto hash;
+ else
+ goto post_hash;
+ case PHJ_PHASE_RESIZING:
+ /* Can't help with serial phase. */
+ goto post_resize;
+ case PHJ_PHASE_REBUCKETING:
+ /* Rebucketing is in progress. Let's help do that. */
+ goto rebucket;
+ default:
+ /* The hash table building work is already finished. */
+ goto finish;
+ }
+ }
+
+ hash:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Make sure our local hashtable is up-to-date so we can hash. */
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_HASHING);
+ ExecHashUpdate(hashtable);
+ }
+
/*
* set expression context
*/
@@ -123,22 +291,98 @@ MultiExecHash(HashState *node)
else
{
/* Not subject to skew optimization, so insert normally */
- ExecHashTableInsert(hashtable, slot, hashvalue);
+ ExecHashTableInsert(hashtable, slot, hashvalue, false);
}
- hashtable->totalTuples += 1;
+ /*
+ * Shared tuple counters are managed by dense_alloc_shared. For
+ * private hash tables we maintain the counter here.
+ */
+ if (!HashJoinTableIsShared(hashtable))
+ hashtable->totalTuples += 1;
}
}
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Update shared tuple count for the current chunk. Other chunks are
+ * accounted for already, when new chunks are allocated.
+ */
+ if (hashtable->primary_chunk != NULL)
+ add_tuple_count(hashtable, hashtable->primary_chunk->ntuples,
+ false);
+ }
+
+ post_hash:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ bool elected_to_resize;
+
+ /*
+ * Wait for all backends to finish hashing. If only one worker is
+ * running the hashing phase because of a non-partial inner plan, the
+ * other workers will pile up here waiting. If multiple worker are
+ * hashing, they should finish close to each other in time.
+ */
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_HASHING);
+ elected_to_resize = BarrierWait(barrier, WAIT_EVENT_HASH_HASHING);
+ /*
+ * Resizing is a serial phase. All but one should skip ahead to
+ * rebucketing, but all workers should update their copy of the shared
+ * tuple count with the final total first.
+ */
+ hashtable->totalTuples =
+ pg_atomic_read_u64(&hashtable->shared->total_primary_tuples);
+ if (!elected_to_resize)
+ goto post_resize;
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_RESIZING);
+ }
+
/* resize the hash table if needed (NTUP_PER_BUCKET exceeded) */
- if (hashtable->nbuckets != hashtable->nbuckets_optimal)
- ExecHashIncreaseNumBuckets(hashtable);
+ ExecHashIncreaseNumBuckets(hashtable);
+
+ post_resize:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_RESIZING);
+ BarrierWait(&hashtable->shared->barrier,
+ WAIT_EVENT_HASH_RESIZING);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_REBUCKETING);
+ }
+
+ rebucket:
+ /* If the table was resized, insert tuples into the new buckets. */
+ ExecHashUpdate(hashtable);
+ ExecHashRebucket(hashtable);
/* Account for the buckets in spaceUsed (reported in EXPLAIN ANALYZE) */
- hashtable->spaceUsed += hashtable->nbuckets * sizeof(HashJoinTuple);
+ hashtable->spaceUsed += hashtable->nbuckets * sizeof(HashJoinBucketHead);
if (hashtable->spaceUsed > hashtable->spacePeak)
hashtable->spacePeak = hashtable->spaceUsed;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_REBUCKETING);
+ BarrierWait(barrier, WAIT_EVENT_HASH_REBUCKETING);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_PROBING);
+ }
+
+ finish:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * All hashing work has finished. The other workers may be probing or
+ * processing unmatched tuples for the initial batch, or dealing with
+ * later batches. The next synchronization point is in ExecHashJoin's
+ * HJ_BUILD_HASHTABLE case, which will figure that out and synchronize
+ * its local state machine with the parallel processing group's phase.
+ */
+ Assert(BarrierPhase(barrier) >= PHJ_PHASE_PROBING);
+ ExecHashUpdate(hashtable);
+ }
+
/* must provide our own instrumentation support */
+ /* TODO: report only the tuples that WE hashed here? */
if (node->ps.instrument)
InstrStopNode(node->ps.instrument, hashtable->totalTuples);
@@ -243,8 +487,9 @@ ExecEndHash(HashState *node)
* ----------------------------------------------------------------
*/
HashJoinTable
-ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
+ExecHashTableCreate(HashState *state, List *hashOperators, bool keepNulls)
{
+ Hash *node;
HashJoinTable hashtable;
Plan *outerNode;
int nbuckets;
@@ -261,6 +506,7 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
* "outer" subtree of this node, but the inner relation of the hashjoin).
* Compute the appropriate size of the hash table.
*/
+ node = (Hash *) state->ps.plan;
outerNode = outerPlan(node);
ExecChooseHashTableSize(outerNode->plan_rows, outerNode->plan_width,
@@ -305,7 +551,14 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
hashtable->spaceUsedSkew = 0;
hashtable->spaceAllowedSkew =
hashtable->spaceAllowed * SKEW_WORK_MEM_PERCENT / 100;
- hashtable->chunks = NULL;
+ hashtable->primary_chunk = NULL;
+ hashtable->secondary_chunk = NULL;
+ hashtable->chunks_to_rebucket = NULL;
+ hashtable->primary_chunk_shared = InvalidDsaPointer;
+ hashtable->secondary_chunk_shared = InvalidDsaPointer;
+ hashtable->area = state->ps.state->es_query_area;
+ hashtable->shared = state->shared_table_data;
+ hashtable->detached_early = false;
#ifdef HJDEBUG
printf("Hashjoin %p: initial nbatch = %d, nbuckets = %d\n",
@@ -368,23 +621,101 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
PrepareTempTablespaces();
}
- /*
- * Prepare context for the first-scan space allocations; allocate the
- * hashbucket array therein, and set each bucket "empty".
- */
- MemoryContextSwitchTo(hashtable->batchCxt);
+ MemoryContextSwitchTo(oldcxt);
- hashtable->buckets = (HashJoinTuple *)
- palloc0(nbuckets * sizeof(HashJoinTuple));
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier;
- /*
- * Set up for skew optimization, if possible and there's a need for more
- * than one batch. (In a one-batch join, there's no point in it.)
- */
- if (nbatch > 1)
- ExecHashBuildSkewHash(hashtable, node, num_skew_mcvs);
+ /*
+ * Attach to the barrier. The corresponding detach operation is in
+ * ExecHashTableDestroy.
+ */
+ barrier = &hashtable->shared->barrier;
+ hashtable->attached_at_phase = BarrierAttach(barrier);
- MemoryContextSwitchTo(oldcxt);
+ /*
+ * So far we have no idea whether there are any other participants, and
+ * if so, what phase they are working on. The only thing we care about
+ * at this point is whether someone has already created the shared
+ * hash table yet. If not, one backend will be elected to do that
+ * now.
+ */
+ if (BarrierPhase(barrier) == PHJ_PHASE_INIT)
+ {
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_INIT))
+ {
+ /* Serial phase: create the hash tables */
+ Size bytes;
+ HashJoinBucketHead *buckets;
+ int i;
+ SharedHashJoinTable shared;
+ dsa_area *area;
+
+ shared = hashtable->shared;
+ area = hashtable->area;
+ bytes = nbuckets * sizeof(HashJoinBucketHead);
+
+ /* Allocate the primary and secondary hash tables. */
+ shared->primary_buckets = dsa_allocate(area, bytes);
+ shared->secondary_buckets = dsa_allocate(area, bytes);
+ if (!DsaPointerIsValid(shared->primary_buckets) ||
+ !DsaPointerIsValid(shared->secondary_buckets))
+ ereport(ERROR,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("out of memory")));
+
+ /* Set up primary table's buckets. */
+ buckets = dsa_get_address(area, shared->primary_buckets);
+ for (i = 0; i < nbuckets; ++i)
+ dsa_pointer_atomic_init(&buckets[i].shared,
+ InvalidDsaPointer);
+ /* Set up secondary table's buckets. */
+ buckets = dsa_get_address(area, shared->secondary_buckets);
+ for (i = 0; i < nbuckets; ++i)
+ dsa_pointer_atomic_init(&buckets[i].shared,
+ InvalidDsaPointer);
+
+ /* Initialize the rest of parallel_state. */
+ hashtable->shared->nbuckets = nbuckets;
+ pg_atomic_write_u32(&hashtable->shared->next_unmatched_bucket,
+ 0);
+ ExecHashJoinRewindBatches(hashtable, 0);
+
+ /* TODO: ExecHashBuildSkewHash */
+
+ /*
+ * The backend-local pointers in hashtable will be set up by
+ * ExecHashUpdate, at each point where they might have
+ * changed.
+ */
+ }
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_CREATING);
+ /* The next synchronization point is in MultiExecHash. */
+ }
+ }
+ else
+ {
+ /*
+ * Prepare context for the first-scan space allocations; allocate the
+ * hashbucket array therein, and set each bucket "empty".
+ */
+ MemoryContextSwitchTo(hashtable->batchCxt);
+
+ hashtable->buckets = (HashJoinBucketHead *)
+ palloc0(nbuckets * sizeof(HashJoinBucketHead));
+
+ MemoryContextSwitchTo(oldcxt);
+
+ /*
+ * Set up for skew optimization, if possible and there's a need for
+ * more than one batch. (In a one-batch join, there's no point in
+ * it.)
+ */
+ if (nbatch > 1)
+ ExecHashBuildSkewHash(hashtable, node, num_skew_mcvs);
+ }
return hashtable;
}
@@ -481,8 +812,8 @@ ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
* Note that both nbuckets and nbatch must be powers of 2 to make
* ExecHashGetBucketAndBatch fast.
*/
- max_pointers = (work_mem * 1024L) / sizeof(HashJoinTuple);
- max_pointers = Min(max_pointers, MaxAllocSize / sizeof(HashJoinTuple));
+ max_pointers = (work_mem * 1024L) / sizeof(HashJoinBucketHead);
+ max_pointers = Min(max_pointers, MaxAllocSize / sizeof(HashJoinBucketHead));
/* If max_pointers isn't a power of 2, must round it down to one */
mppow2 = 1L << my_log2(max_pointers);
if (max_pointers != mppow2)
@@ -504,7 +835,7 @@ ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
* If there's not enough space to store the projected number of tuples and
* the required bucket headers, we will need multiple batches.
*/
- bucket_bytes = sizeof(HashJoinTuple) * nbuckets;
+ bucket_bytes = sizeof(HashJoinBucketHead) * nbuckets;
if (inner_rel_bytes + bucket_bytes > hash_table_bytes)
{
/* We'll need multiple batches */
@@ -519,12 +850,12 @@ ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
* NTUP_PER_BUCKET tuples, whose projected size already includes
* overhead for the hash code, pointer to the next tuple, etc.
*/
- bucket_size = (tupsize * NTUP_PER_BUCKET + sizeof(HashJoinTuple));
+ bucket_size = (tupsize * NTUP_PER_BUCKET + sizeof(HashJoinBucketHead));
lbuckets = 1L << my_log2(hash_table_bytes / bucket_size);
lbuckets = Min(lbuckets, max_pointers);
nbuckets = (int) lbuckets;
nbuckets = 1 << my_log2(nbuckets);
- bucket_bytes = nbuckets * sizeof(HashJoinTuple);
+ bucket_bytes = nbuckets * sizeof(HashJoinBucketHead);
/*
* Buckets are simple pointers to hashjoin tuples, while tupsize
@@ -564,6 +895,38 @@ ExecHashTableDestroy(HashJoinTable hashtable)
{
int i;
+ /* Detached, if we haven't already. */
+ if (HashJoinTableIsShared(hashtable) && !hashtable->detached_early)
+ {
+ Barrier *barrier = &hashtable->shared->barrier;
+
+ /*
+ * TODO: Can we just detach if there is only one batch, but wait here
+ * if there is more than one (to make sure batch files created by this
+ * participant are not deleted)? When detaching, the last one to
+ * detach should do the cleanup work, and/or leave things in the right
+ * state for rescanning.
+ */
+
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_DESTROY))
+ {
+ /* Serial: free the tables */
+ if (DsaPointerIsValid(hashtable->shared->primary_buckets))
+ {
+ dsa_free(hashtable->area,
+ hashtable->shared->primary_buckets);
+ hashtable->shared->primary_buckets = InvalidDsaPointer;
+ }
+ if (DsaPointerIsValid(hashtable->shared->secondary_buckets))
+ {
+ dsa_free(hashtable->area,
+ hashtable->shared->secondary_buckets);
+ hashtable->shared->secondary_buckets = InvalidDsaPointer;
+ }
+ }
+ BarrierDetach(&hashtable->shared->barrier);
+ }
+
/*
* Make sure all the temp files are closed. We skip batch 0, since it
* can't have any temp files (and the arrays might not even exist if
@@ -600,6 +963,15 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
long nfreed;
HashMemoryChunk oldchunks;
+ /*
+ * TODO: Implement for shared tables. It's OK for different workers to
+ * have different ideas of nbatch for short times, as long as they agree
+ * at key points in time (ie when deciding if we're finished). Working on
+ * this...
+ */
+ if (HashJoinTableIsShared(hashtable))
+ return;
+
/* do nothing if we've decided to shut off growth */
if (!hashtable->growEnabled)
return;
@@ -661,7 +1033,7 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
hashtable->log2_nbuckets = hashtable->log2_nbuckets_optimal;
hashtable->buckets = repalloc(hashtable->buckets,
- sizeof(HashJoinTuple) * hashtable->nbuckets);
+ sizeof(HashJoinBucketHead) * hashtable->nbuckets);
}
/*
@@ -669,14 +1041,14 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
* buckets now and not have to keep track which tuples in the buckets have
* already been processed. We will free the old chunks as we go.
*/
- memset(hashtable->buckets, 0, sizeof(HashJoinTuple) * hashtable->nbuckets);
- oldchunks = hashtable->chunks;
- hashtable->chunks = NULL;
+ memset(hashtable->buckets, 0, sizeof(HashJoinBucketHead) * hashtable->nbuckets);
+ oldchunks = hashtable->primary_chunk;
+ hashtable->primary_chunk = NULL;
/* so, let's scan through the old chunks, and all tuples in each chunk */
while (oldchunks != NULL)
{
- HashMemoryChunk nextchunk = oldchunks->next;
+ HashMemoryChunk nextchunk = oldchunks->next.private;
/* position within the buffer (up to oldchunks->used) */
size_t idx = 0;
@@ -699,20 +1071,23 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
/* keep tuple in memory - copy it into the new chunk */
HashJoinTuple copyTuple;
- copyTuple = (HashJoinTuple) dense_alloc(hashtable, hashTupleSize);
+ copyTuple = (HashJoinTuple)
+ dense_alloc(hashtable, hashTupleSize);
memcpy(copyTuple, hashTuple, hashTupleSize);
/* and add it back to the appropriate bucket */
- copyTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = copyTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, copyTuple,
+ InvalidDsaPointer);
}
else
{
/* dump it out */
Assert(batchno > curbatch);
- ExecHashJoinSaveTuple(HJTUPLE_MINTUPLE(hashTuple),
+ ExecHashJoinSaveTuple(hashtable,
+ HJTUPLE_MINTUPLE(hashTuple),
hashTuple->hashvalue,
- &hashtable->innerBatchFile[batchno]);
+ batchno,
+ true);
hashtable->spaceUsed -= hashTupleSize;
nfreed++;
@@ -758,8 +1133,6 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
static void
ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
{
- HashMemoryChunk chunk;
-
/* do nothing if not an increase (it's called increase for a reason) */
if (hashtable->nbuckets >= hashtable->nbuckets_optimal)
return;
@@ -780,16 +1153,156 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
* Just reallocate the proper number of buckets - we don't need to walk
* through them - we can walk the dense-allocated chunks (just like in
* ExecHashIncreaseNumBatches, but without all the copying into new
- * chunks)
+ * chunks): see ExecHashRebucket, which must be called next.
*/
- hashtable->buckets =
- (HashJoinTuple *) repalloc(hashtable->buckets,
- hashtable->nbuckets * sizeof(HashJoinTuple));
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Size bytes;
+ int i;
+
+ /* Serial phase: only one backend reallocates. */
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_RESIZING);
+
+ /* Free the old arrays. */
+ dsa_free(hashtable->area,
+ hashtable->shared->primary_buckets);
+ dsa_free(hashtable->area,
+ hashtable->shared->secondary_buckets);
+ /* Allocate replacements. */
+ bytes = hashtable->nbuckets * sizeof(HashJoinBucketHead);
+ hashtable->shared->primary_buckets =
+ dsa_allocate(hashtable->area, bytes);
+ hashtable->shared->secondary_buckets =
+ dsa_allocate(hashtable->area, bytes);
+ if (!DsaPointerIsValid(hashtable->shared->primary_buckets) ||
+ !DsaPointerIsValid(hashtable->shared->secondary_buckets))
+ ereport(ERROR,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("out of memory")));
+ /* Initialize empty buckets. */
+ hashtable->buckets =
+ dsa_get_address(hashtable->area,
+ hashtable->shared->primary_buckets);
+ for (i = 0; i < hashtable->nbuckets; ++i)
+ dsa_pointer_atomic_write(&hashtable->buckets[i].shared,
+ InvalidDsaPointer);
+ hashtable->next_buckets =
+ dsa_get_address(hashtable->area,
+ hashtable->shared->secondary_buckets);
+ for (i = 0; i < hashtable->nbuckets; ++i)
+ dsa_pointer_atomic_write(&hashtable->next_buckets[i].shared,
+ InvalidDsaPointer);
+ hashtable->shared->nbuckets = hashtable->nbuckets;
+ /* Move all primary chunks to the rebucket list. */
+ dsa_pointer_atomic_write(&hashtable->shared->chunks_to_rebucket,
+ dsa_pointer_atomic_read(&hashtable->shared->head_primary_chunk));
+ dsa_pointer_atomic_write(&hashtable->shared->head_primary_chunk,
+ InvalidDsaPointer);
+ }
+ else
+ {
+ hashtable->buckets =
+ (HashJoinBucketHead *) repalloc(hashtable->buckets,
+ hashtable->nbuckets * sizeof(HashJoinBucketHead));
+
+ memset(hashtable->buckets, 0, hashtable->nbuckets * sizeof(HashJoinBucketHead));
+ /* Move all chunks to the rebucket list. */
+ hashtable->chunks_to_rebucket = hashtable->primary_chunk;
+ hashtable->primary_chunk = NULL;
+ }
+}
+
+/*
+ * Pop a memory chunk from a given list atomically. Returns a backend-local
+ * pointer to the chunk, or NULL if the list is empty. Also sets *chunk_out
+ * to the dsa_pointer to the chunk.
+ */
+static HashMemoryChunk
+ExecHashPopChunk(HashJoinTable hashtable,
+ dsa_pointer *chunk_out,
+ dsa_pointer_atomic *head)
+{
+ HashMemoryChunk chunk = NULL;
+
+ /*
+ * We could see a stale empty list and exist early without a barrier, so
+ * explicitly include one before we read the head of the list for the
+ * first time.
+ */
+ pg_read_barrier();
- memset(hashtable->buckets, 0, hashtable->nbuckets * sizeof(HashJoinTuple));
+ for (;;)
+ {
+ *chunk_out = dsa_pointer_atomic_read(head);
+ if (!DsaPointerIsValid(*chunk_out))
+ {
+ chunk = NULL;
+ break;
+ }
+ chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, *chunk_out);
+ if (dsa_pointer_atomic_compare_exchange(head,
+ chunk_out,
+ chunk->next.shared))
+ break;
+ }
+
+ return chunk;
+}
+
+/*
+ * Push a shared memory chunk onto a given list atomically.
+ */
+static void
+ExecHashPushChunk(HashJoinTable hashtable,
+ HashMemoryChunk chunk,
+ dsa_pointer chunk_shared,
+ dsa_pointer_atomic *head)
+{
+ Assert(chunk == dsa_get_address(hashtable->area, chunk_shared));
+
+ for (;;)
+ {
+ chunk->next.shared = dsa_pointer_atomic_read(head);
+ if (dsa_pointer_atomic_compare_exchange(head,
+ &chunk->next.shared,
+ chunk_shared))
+ break;
+ }
+}
+
+/*
+ * ExecHashRebucket
+ * insert the tuples from all chunks into the correct bucket
+ */
+static void
+ExecHashRebucket(HashJoinTable hashtable)
+{
+ HashMemoryChunk chunk;
+ dsa_pointer chunk_shared;
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * This is a parallel phase. Workers will atomically pop one chunk at
+ * a time and rebucket all of its tuples.
+ */
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_REBUCKETING);
+ }
- /* scan through all tuples in all chunks to rebuild the hash table */
- for (chunk = hashtable->chunks; chunk != NULL; chunk = chunk->next)
+ /*
+ * Scan through all tuples in all chunks in the rebucket list to rebuild
+ * the hash table.
+ */
+ if (HashJoinTableIsShared(hashtable))
+ chunk =
+ ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_to_rebucket);
+ else
+ chunk = hashtable->chunks_to_rebucket;
+ while (chunk != NULL)
{
/* process all tuples stored in this chunk */
size_t idx = 0;
@@ -797,6 +1310,8 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
while (idx < chunk->used)
{
HashJoinTuple hashTuple = (HashJoinTuple) (chunk->data + idx);
+ dsa_pointer hashTuple_shared = chunk_shared +
+ offsetof(HashMemoryChunkData, data) + idx;
int bucketno;
int batchno;
@@ -804,16 +1319,52 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
&bucketno, &batchno);
/* add the tuple to the proper bucket */
- hashTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = hashTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, hashTuple,
+ hashTuple_shared);
/* advance index past the tuple */
idx += MAXALIGN(HJTUPLE_OVERHEAD +
HJTUPLE_MINTUPLE(hashTuple)->t_len);
}
+
+ /* Push chunk onto regular list and move to next chunk. */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ &hashtable->shared->head_primary_chunk);
+ chunk =
+ ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_to_rebucket);
+ }
+ else
+ {
+ HashMemoryChunk next = chunk->next.private;
+
+ chunk->next.private = hashtable->primary_chunk;
+ hashtable->primary_chunk = chunk;
+ chunk = next;
+ }
}
}
+static void
+ExecHashTableComputeOptimalBuckets(HashJoinTable hashtable)
+{
+ double ntuples = (hashtable->totalTuples - hashtable->skewTuples);
+
+ /*
+ * Guard against integer overflow and alloc size overflow. The
+ * MaxAllocSize limitation doesn't really apply for shared hash tables,
+ * since DSA has no such limit, but for now let's apply the same limit.
+ */
+ while (ntuples > (hashtable->nbuckets_optimal * NTUP_PER_BUCKET) &&
+ hashtable->nbuckets_optimal <= INT_MAX / 2 &&
+ hashtable->nbuckets_optimal * 2 <= MaxAllocSize / sizeof(HashJoinBucketHead))
+ {
+ hashtable->nbuckets_optimal *= 2;
+ hashtable->log2_nbuckets_optimal += 1;
+ }
+}
/*
* ExecHashTableInsert
@@ -829,7 +1380,8 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
void
ExecHashTableInsert(HashJoinTable hashtable,
TupleTableSlot *slot,
- uint32 hashvalue)
+ uint32 hashvalue,
+ bool secondary)
{
MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot);
int bucketno;
@@ -848,11 +1400,17 @@ ExecHashTableInsert(HashJoinTable hashtable,
*/
HashJoinTuple hashTuple;
int hashTupleSize;
- double ntuples = (hashtable->totalTuples - hashtable->skewTuples);
+ dsa_pointer hashTuple_shared = InvalidDsaPointer;
/* Create the HashJoinTuple */
hashTupleSize = HJTUPLE_OVERHEAD + tuple->t_len;
- hashTuple = (HashJoinTuple) dense_alloc(hashtable, hashTupleSize);
+ if (HashJoinTableIsShared(hashtable))
+ hashTuple = (HashJoinTuple)
+ dense_alloc_shared(hashtable, hashTupleSize,
+ &hashTuple_shared, secondary);
+ else
+ hashTuple = (HashJoinTuple)
+ dense_alloc(hashtable, hashTupleSize);
hashTuple->hashvalue = hashvalue;
memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
@@ -866,32 +1424,23 @@ ExecHashTableInsert(HashJoinTable hashtable,
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple));
/* Push it onto the front of the bucket's list */
- hashTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = hashTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, hashTuple,
+ hashTuple_shared);
/*
* Increase the (optimal) number of buckets if we just exceeded the
* NTUP_PER_BUCKET threshold, but only when there's still a single
* batch.
*/
- if (hashtable->nbatch == 1 &&
- ntuples > (hashtable->nbuckets_optimal * NTUP_PER_BUCKET))
- {
- /* Guard against integer overflow and alloc size overflow */
- if (hashtable->nbuckets_optimal <= INT_MAX / 2 &&
- hashtable->nbuckets_optimal * 2 <= MaxAllocSize / sizeof(HashJoinTuple))
- {
- hashtable->nbuckets_optimal *= 2;
- hashtable->log2_nbuckets_optimal += 1;
- }
- }
+ if (hashtable->nbatch == 1)
+ ExecHashTableComputeOptimalBuckets(hashtable);
/* Account for space used, and back off if we've used too much */
hashtable->spaceUsed += hashTupleSize;
if (hashtable->spaceUsed > hashtable->spacePeak)
hashtable->spacePeak = hashtable->spaceUsed;
if (hashtable->spaceUsed +
- hashtable->nbuckets_optimal * sizeof(HashJoinTuple)
+ hashtable->nbuckets_optimal * sizeof(HashJoinBucketHead)
> hashtable->spaceAllowed)
ExecHashIncreaseNumBatches(hashtable);
}
@@ -901,9 +1450,11 @@ ExecHashTableInsert(HashJoinTable hashtable,
* put the tuple into a temp file for later batches
*/
Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(tuple,
+ ExecHashJoinSaveTuple(hashtable,
+ tuple,
hashvalue,
- &hashtable->innerBatchFile[batchno]);
+ batchno,
+ true);
}
}
@@ -1047,6 +1598,138 @@ ExecHashGetBucketAndBatch(HashJoinTable hashtable,
}
/*
+ * Update the local hashtable with the current pointers and sizes from
+ * hashtable->parallel_state.
+ */
+void
+ExecHashUpdate(HashJoinTable hashtable)
+{
+ Barrier *barrier;
+
+ if (!HashJoinTableIsShared(hashtable))
+ return;
+
+ barrier = &hashtable->shared->barrier;
+
+ /*
+ * This should only be called in a phase when the hash table is not being
+ * mutated (ie resized, swapped etc).
+ */
+ Assert(!PHJ_PHASE_MUTATING_TABLE(
+ BarrierPhase(&hashtable->shared->barrier)));
+
+ /* The primary hash table. */
+ hashtable->buckets = (HashJoinBucketHead *)
+ dsa_get_address(hashtable->area,
+ hashtable->shared->primary_buckets);
+ hashtable->nbuckets = hashtable->shared->nbuckets;
+ hashtable->log2_nbuckets = my_log2(hashtable->nbuckets);
+ /* The secondary hash table, if there is one (NULL for initial batch). */
+ hashtable->next_buckets = (HashJoinBucketHead *)
+ dsa_get_address(hashtable->area,
+ hashtable->shared->secondary_buckets);
+
+ hashtable->curbatch = PHJ_PHASE_TO_BATCHNO(BarrierPhase(barrier));
+}
+
+/*
+ * Get the next tuple in the same bucket as 'tuple'.
+ */
+static HashJoinTuple
+next_tuple_in_bucket(HashJoinTable table, HashJoinTuple tuple)
+{
+ if (HashJoinTableIsShared(table))
+ return (HashJoinTuple)
+ dsa_get_address(table->area, tuple->next.shared);
+ else
+ return tuple->next.private;
+}
+
+/*
+ * Get the first tuple in a given skew bucket identified by number.
+ */
+static HashJoinTuple
+first_tuple_in_skew_bucket(HashJoinTable table, int skew_bucket_no)
+{
+ if (HashJoinTableIsShared(table))
+ return (HashJoinTuple)
+ dsa_get_address(table->area,
+ table->skewBucket[skew_bucket_no]->tuples.shared);
+ else
+ return table->skewBucket[skew_bucket_no]->tuples.private;
+}
+
+/*
+ * Get the first tuple in a given bucket identified by number.
+ */
+static HashJoinTuple
+first_tuple_in_bucket(HashJoinTable table, int bucket_no)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ dsa_pointer p =
+ dsa_pointer_atomic_read(&table->buckets[bucket_no].shared);
+ return (HashJoinTuple) dsa_get_address(table->area, p);
+ }
+ else
+ return table->buckets[bucket_no].private;
+}
+
+/*
+ * Insert a tuple at the front of a given bucket identified by number. For
+ * shared hash joins, tuple_shared must be provided, pointing to the tuple in
+ * the dsa_area backing the table. For private hash joins, it should be
+ * InvalidDsaPointer.
+ */
+static void
+insert_tuple_into_bucket(HashJoinTable table, int bucket_no,
+ HashJoinTuple tuple, dsa_pointer tuple_shared)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ Assert(tuple == dsa_get_address(table->area, tuple_shared));
+ for (;;)
+ {
+ tuple->next.shared =
+ dsa_pointer_atomic_read(&table->buckets[bucket_no].shared);
+ if (dsa_pointer_atomic_compare_exchange(&table->buckets[bucket_no].shared,
+ &tuple->next.shared,
+ tuple_shared))
+ break;
+ }
+ }
+ else
+ {
+ tuple->next.private = table->buckets[bucket_no].private;
+ table->buckets[bucket_no].private = tuple;
+ }
+}
+
+/*
+ * Insert a tuple at the front of a given skew bucket identified by number.
+ * For shared hash joins, tuple_shared must be provided, pointing to the tuple
+ * in the dsa_area backing the table. For private hash joins, it should be
+ * InvalidDsaPointer.
+ */
+static void
+insert_tuple_into_skew_bucket(HashJoinTable table, int skew_bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_shared)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ tuple->next.shared =
+ table->skewBucket[skew_bucket_no]->tuples.shared;
+ table->skewBucket[skew_bucket_no]->tuples.shared = tuple_shared;
+ }
+ else
+ {
+ tuple->next.private = table->skewBucket[skew_bucket_no]->tuples.private;
+ table->skewBucket[skew_bucket_no]->tuples.private = tuple;
+ }
+}
+
+/*
* ExecScanHashBucket
* scan a hash bucket for matches to the current outer tuple
*
@@ -1073,11 +1756,12 @@ ExecScanHashBucket(HashJoinState *hjstate,
* otherwise scan the standard hashtable bucket.
*/
if (hashTuple != NULL)
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
else if (hjstate->hj_CurSkewBucketNo != INVALID_SKEW_BUCKET_NO)
- hashTuple = hashtable->skewBucket[hjstate->hj_CurSkewBucketNo]->tuples;
+ hashTuple = first_tuple_in_skew_bucket(hashtable,
+ hjstate->hj_CurSkewBucketNo);
else
- hashTuple = hashtable->buckets[hjstate->hj_CurBucketNo];
+ hashTuple = first_tuple_in_bucket(hashtable, hjstate->hj_CurBucketNo);
while (hashTuple != NULL)
{
@@ -1101,7 +1785,7 @@ ExecScanHashBucket(HashJoinState *hjstate,
}
}
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
}
/*
@@ -1144,6 +1828,21 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
HashJoinTable hashtable = hjstate->hj_HashTable;
HashJoinTuple hashTuple = hjstate->hj_CurTuple;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ int phase PG_USED_FOR_ASSERTS_ONLY;
+
+ /*
+ * TODO: This walks the buckets in parallel mode, like the existing
+ * code, but it might make more sense to hand out chunks to workers
+ * instead of buckets.
+ */
+
+ phase = BarrierPhase(&hashtable->shared->barrier);
+ Assert(PHJ_PHASE_TO_SUBPHASE(phase) == PHJ_SUBPHASE_UNMATCHED);
+ Assert(PHJ_PHASE_TO_BATCHNO(phase) == hashtable->curbatch);
+ }
+
for (;;)
{
/*
@@ -1152,21 +1851,35 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
* bucket.
*/
if (hashTuple != NULL)
- hashTuple = hashTuple->next;
- else if (hjstate->hj_CurBucketNo < hashtable->nbuckets)
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
+ else if (HashJoinTableIsShared(hashtable))
{
- hashTuple = hashtable->buckets[hjstate->hj_CurBucketNo];
- hjstate->hj_CurBucketNo++;
+ int bucketno =
+ (int) pg_atomic_fetch_add_u32(
+ &hashtable->shared->next_unmatched_bucket, 1);
+
+ if (bucketno >= hashtable->nbuckets)
+ break; /* finished all buckets */
+
+ hashTuple = first_tuple_in_bucket(hashtable, bucketno);
+
+ /* TODO: parallel skew bucket support */
}
- else if (hjstate->hj_CurSkewBucketNo < hashtable->nSkewBuckets)
+ else
{
- int j = hashtable->skewBucketNums[hjstate->hj_CurSkewBucketNo];
+ if (hjstate->hj_CurBucketNo < hashtable->nbuckets)
+ hashTuple = first_tuple_in_bucket(hashtable,
+ hjstate->hj_CurBucketNo++);
+ else if (hjstate->hj_CurSkewBucketNo < hashtable->nSkewBuckets)
+ {
+ int j = hashtable->skewBucketNums[hjstate->hj_CurSkewBucketNo];
- hashTuple = hashtable->skewBucket[j]->tuples;
- hjstate->hj_CurSkewBucketNo++;
+ hashTuple = first_tuple_in_skew_bucket(hashtable, j);
+ hjstate->hj_CurSkewBucketNo++;
+ }
+ else
+ break; /* finished all buckets */
}
- else
- break; /* finished all buckets */
while (hashTuple != NULL)
{
@@ -1191,7 +1904,7 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
return true;
}
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
}
}
@@ -1212,6 +1925,65 @@ ExecHashTableReset(HashJoinTable hashtable)
MemoryContext oldcxt;
int nbuckets = hashtable->nbuckets;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Wait for all workers to finish accessing the primary hash table. */
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_UNMATCHED);
+ if (BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASH_UNMATCHED))
+ {
+ /* Serial phase: promote the secondary table to primary. */
+ dsa_pointer tmp;
+ int i;
+
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_PROMOTING);
+
+ /* Clear the old primary table. */
+ for (i = 0; i < nbuckets; ++i)
+ dsa_pointer_atomic_write(&hashtable->buckets[i].shared,
+ InvalidDsaPointer);
+
+ /* Swap the two tables. */
+ tmp = hashtable->shared->primary_buckets;
+ hashtable->shared->primary_buckets =
+ hashtable->shared->secondary_buckets;
+ hashtable->shared->secondary_buckets = tmp;
+
+ /* Swap the chunk lists. */
+ tmp = dsa_pointer_atomic_read(&hashtable->shared->head_primary_chunk);
+ dsa_pointer_atomic_write(&hashtable->shared->head_primary_chunk,
+ dsa_pointer_atomic_read(&hashtable->shared->head_secondary_chunk));
+ dsa_pointer_atomic_write(&hashtable->shared->head_secondary_chunk,
+ tmp);
+
+ /* Free the secondary chunks. */
+ /* TODO: Or put them on a freelist in one cheap operation instead? */
+ tmp = dsa_pointer_atomic_read(&hashtable->shared->head_secondary_chunk);
+ while (DsaPointerIsValid(tmp))
+ {
+ HashMemoryChunk chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, tmp);
+ dsa_pointer next = chunk->next.shared;
+
+ dsa_free(hashtable->area, tmp);
+ tmp = next;
+ }
+ dsa_pointer_atomic_write(&hashtable->shared->head_secondary_chunk,
+ InvalidDsaPointer);
+
+ /* Reset the unmatched cursor. */
+ pg_atomic_write_u32(&hashtable->shared->next_unmatched_bucket,
+ 0);
+ }
+ /* Wait again, so that all workers now have the new table. */
+ BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASH_PROMOTING);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_LOADING);
+ ExecHashUpdate(hashtable);
+ return;
+ }
+
/*
* Release all the hash buckets and tuples acquired in the prior pass, and
* reinitialize the context for a new pass.
@@ -1220,15 +1992,15 @@ ExecHashTableReset(HashJoinTable hashtable)
oldcxt = MemoryContextSwitchTo(hashtable->batchCxt);
/* Reallocate and reinitialize the hash bucket headers. */
- hashtable->buckets = (HashJoinTuple *)
- palloc0(nbuckets * sizeof(HashJoinTuple));
+ hashtable->buckets = (HashJoinBucketHead *)
+ palloc0(nbuckets * sizeof(HashJoinBucketHead));
hashtable->spaceUsed = 0;
MemoryContextSwitchTo(oldcxt);
/* Forget the chunks (the memory was freed by the context reset above). */
- hashtable->chunks = NULL;
+ hashtable->primary_chunk = NULL;
}
/*
@@ -1241,10 +2013,14 @@ ExecHashTableResetMatchFlags(HashJoinTable hashtable)
HashJoinTuple tuple;
int i;
+ /* TODO: share parallel reset work! coordinate! */
+
/* Reset all flags in the main table ... */
for (i = 0; i < hashtable->nbuckets; i++)
{
- for (tuple = hashtable->buckets[i]; tuple != NULL; tuple = tuple->next)
+ for (tuple = first_tuple_in_bucket(hashtable, i);
+ tuple != NULL;
+ tuple = next_tuple_in_bucket(hashtable, tuple))
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(tuple));
}
@@ -1252,9 +2028,10 @@ ExecHashTableResetMatchFlags(HashJoinTable hashtable)
for (i = 0; i < hashtable->nSkewBuckets; i++)
{
int j = hashtable->skewBucketNums[i];
- HashSkewBucket *skewBucket = hashtable->skewBucket[j];
- for (tuple = skewBucket->tuples; tuple != NULL; tuple = tuple->next)
+ for (tuple = first_tuple_in_skew_bucket(hashtable, j);
+ tuple != NULL;
+ tuple = next_tuple_in_bucket(hashtable, tuple))
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(tuple));
}
}
@@ -1414,11 +2191,11 @@ ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node, int mcvsToUse)
continue;
/* Okay, create a new skew bucket for this hashvalue. */
- hashtable->skewBucket[bucket] = (HashSkewBucket *)
+ hashtable->skewBucket[bucket] = (HashSkewBucket *) /* TODO */
MemoryContextAlloc(hashtable->batchCxt,
sizeof(HashSkewBucket));
hashtable->skewBucket[bucket]->hashvalue = hashvalue;
- hashtable->skewBucket[bucket]->tuples = NULL;
+ hashtable->skewBucket[bucket]->tuples.private = NULL;
hashtable->skewBucketNums[hashtable->nSkewBuckets] = bucket;
hashtable->nSkewBuckets++;
hashtable->spaceUsed += SKEW_BUCKET_OVERHEAD;
@@ -1496,18 +2273,29 @@ ExecHashSkewTableInsert(HashJoinTable hashtable,
MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot);
HashJoinTuple hashTuple;
int hashTupleSize;
+ dsa_pointer tuple_pointer;
/* Create the HashJoinTuple */
hashTupleSize = HJTUPLE_OVERHEAD + tuple->t_len;
- hashTuple = (HashJoinTuple) MemoryContextAlloc(hashtable->batchCxt,
- hashTupleSize);
+ if (HashJoinTableIsShared(hashtable))
+ {
+ tuple_pointer = dsa_allocate(hashtable->area, hashTupleSize);
+ hashTuple = (HashJoinTuple) dsa_get_address(hashtable->area,
+ tuple_pointer);
+ }
+ else
+ {
+ tuple_pointer = InvalidDsaPointer;
+ hashTuple = (HashJoinTuple) MemoryContextAlloc(hashtable->batchCxt,
+ hashTupleSize);
+ }
hashTuple->hashvalue = hashvalue;
memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple));
/* Push it onto the front of the skew bucket's list */
- hashTuple->next = hashtable->skewBucket[bucketNumber]->tuples;
- hashtable->skewBucket[bucketNumber]->tuples = hashTuple;
+ insert_tuple_into_skew_bucket(hashtable, bucketNumber, hashTuple,
+ tuple_pointer);
/* Account for space used, and back off if we've used too much */
hashtable->spaceUsed += hashTupleSize;
@@ -1538,6 +2326,9 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
int batchno;
HashJoinTuple hashTuple;
+ /* TODO: skew buckets not yet supported for parallel mode */
+ Assert(!HashJoinTableIsShared(hashtable));
+
/* Locate the bucket to remove */
bucketToRemove = hashtable->skewBucketNums[hashtable->nSkewBuckets - 1];
bucket = hashtable->skewBucket[bucketToRemove];
@@ -1552,10 +2343,10 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno);
/* Process all tuples in the bucket */
- hashTuple = bucket->tuples;
+ hashTuple = first_tuple_in_skew_bucket(hashtable, bucketToRemove);
while (hashTuple != NULL)
{
- HashJoinTuple nextHashTuple = hashTuple->next;
+ HashJoinTuple nextHashTuple = next_tuple_in_bucket(hashtable, hashTuple);
MinimalTuple tuple;
Size tupleSize;
@@ -1581,8 +2372,8 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
memcpy(copyTuple, hashTuple, tupleSize);
pfree(hashTuple);
- copyTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = copyTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, copyTuple,
+ InvalidDsaPointer);
/* We have reduced skew space, but overall space doesn't change */
hashtable->spaceUsedSkew -= tupleSize;
@@ -1591,8 +2382,8 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
{
/* Put the tuple into a temp file for later batches */
Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(tuple, hashvalue,
- &hashtable->innerBatchFile[batchno]);
+ ExecHashJoinSaveTuple(hashtable, tuple, hashvalue,
+ batchno, true);
pfree(hashTuple);
hashtable->spaceUsed -= tupleSize;
hashtable->spaceUsedSkew -= tupleSize;
@@ -1636,6 +2427,141 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
}
/*
+ * Add to the primary or secondary tuple counter.
+ */
+static void
+add_tuple_count(HashJoinTable hashtable, int count, bool secondary)
+{
+ if (secondary)
+ pg_atomic_fetch_add_u64(&hashtable->shared->total_secondary_tuples,
+ count);
+ else
+ {
+ uint32 total =
+ pg_atomic_fetch_add_u64(&hashtable->shared->total_primary_tuples,
+ count);
+ /* Also update this backend's counter. */
+ hashtable->totalTuples = total + count;
+ }
+}
+
+/*
+ * Allocate 'size' bytes from the currently active shared HashMemoryChunk.
+ * This is essentially the same as the private memory version, but allocates
+ * from separate chunks for the secondary table and periodically updates the
+ * shared tuple counter.
+ */
+static void *
+dense_alloc_shared(HashJoinTable hashtable,
+ Size size,
+ dsa_pointer *shared,
+ bool secondary)
+{
+ dsa_pointer chunk_shared;
+ HashMemoryChunk chunk;
+ char *ptr;
+
+ /* just in case the size is not already aligned properly */
+ size = MAXALIGN(size);
+
+ /*
+ * If tuple size is larger than of 1/4 of chunk size, allocate a separate
+ * chunk.
+ */
+ if (size > HASH_CHUNK_THRESHOLD)
+ {
+ /* allocate new chunk */
+ chunk_shared =
+ dsa_allocate(hashtable->area,
+ offsetof(HashMemoryChunkData, data) + size);
+ chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, chunk_shared);
+ *shared = chunk_shared + offsetof(HashMemoryChunkData, data);
+ chunk->maxlen = size;
+ chunk->used = size;
+ chunk->ntuples = 1;
+
+ /*
+ * Push onto the appropriate chunk list, but don't make it the current
+ * chunk because it hasn't got any more useful space in it. The
+ * current chunk may still have space, so keep that one current.
+ */
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ secondary ?
+ &hashtable->shared->head_secondary_chunk :
+ &hashtable->shared->head_primary_chunk);
+
+ /* Count these huge tuples immediately. */
+ add_tuple_count(hashtable, 1, secondary);
+ return chunk->data;
+ }
+
+ /*
+ * See if we have enough space for it in the current chunk (if any). If
+ * not, allocate a fresh chunk.
+ */
+ chunk = secondary ? hashtable->secondary_chunk : hashtable->primary_chunk;
+ if (chunk == NULL || (chunk->maxlen - chunk->used) < size)
+ {
+ /*
+ * Add the tuplecount for the outgoing chunk to the shared counter.
+ * Doing this only every time we need to allocate a new chunk should
+ * reduce contention on the shared counter.
+ */
+ if (chunk != NULL)
+ add_tuple_count(hashtable, chunk->ntuples, secondary);
+
+ /*
+ * Allocate new chunk and make it the current chunk for this backend
+ * to allocate from.
+ */
+ chunk_shared =
+ dsa_allocate(hashtable->area,
+ offsetof(HashMemoryChunkData, data) +
+ HASH_CHUNK_SIZE);
+ chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, chunk_shared);
+ *shared = chunk_shared + offsetof(HashMemoryChunkData, data);
+ if (secondary)
+ {
+ hashtable->secondary_chunk = chunk;
+ hashtable->secondary_chunk_shared = chunk_shared;
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ &hashtable->shared->head_secondary_chunk);
+ }
+ else
+ {
+ hashtable->primary_chunk = chunk;
+ hashtable->primary_chunk_shared = chunk_shared;
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ &hashtable->shared->head_primary_chunk);
+ }
+ chunk->maxlen = HASH_CHUNK_SIZE;
+ chunk->used = size;
+ chunk->ntuples = 1;
+
+ /*
+ * The shared tuple counter will be updated when this chunk is
+ * eventually full. See above.
+ */
+
+ return chunk->data;
+ }
+
+ /* There is enough space in the current chunk, let's add the tuple */
+ chunk_shared =
+ secondary ? hashtable->secondary_chunk_shared :
+ hashtable->primary_chunk_shared;
+ ptr = chunk->data + chunk->used;
+ *shared = chunk_shared + offsetof(HashMemoryChunkData, data) + chunk->used;
+ chunk->used += size;
+ chunk->ntuples += 1;
+
+ /* return pointer to the start of the tuple memory */
+ return ptr;
+}
+
+/*
* Allocate 'size' bytes from the currently active HashMemoryChunk
*/
static void *
@@ -1653,9 +2579,11 @@ dense_alloc(HashJoinTable hashtable, Size size)
*/
if (size > HASH_CHUNK_THRESHOLD)
{
+
/* allocate new chunk and put it at the beginning of the list */
- newChunk = (HashMemoryChunk) MemoryContextAlloc(hashtable->batchCxt,
- offsetof(HashMemoryChunkData, data) + size);
+ newChunk = (HashMemoryChunk)
+ MemoryContextAlloc(hashtable->batchCxt,
+ offsetof(HashMemoryChunkData, data) + size);
newChunk->maxlen = size;
newChunk->used = 0;
newChunk->ntuples = 0;
@@ -1664,15 +2592,15 @@ dense_alloc(HashJoinTable hashtable, Size size)
* Add this chunk to the list after the first existing chunk, so that
* we don't lose the remaining space in the "current" chunk.
*/
- if (hashtable->chunks != NULL)
+ if (hashtable->primary_chunk != NULL)
{
- newChunk->next = hashtable->chunks->next;
- hashtable->chunks->next = newChunk;
+ newChunk->next.private = hashtable->primary_chunk->next.private;
+ hashtable->primary_chunk->next.private = newChunk;
}
else
{
- newChunk->next = hashtable->chunks;
- hashtable->chunks = newChunk;
+ newChunk->next.private = NULL;
+ hashtable->primary_chunk = newChunk;
}
newChunk->used += size;
@@ -1685,27 +2613,27 @@ dense_alloc(HashJoinTable hashtable, Size size)
* See if we have enough space for it in the current chunk (if any). If
* not, allocate a fresh chunk.
*/
- if ((hashtable->chunks == NULL) ||
- (hashtable->chunks->maxlen - hashtable->chunks->used) < size)
+ if ((hashtable->primary_chunk == NULL) ||
+ (hashtable->primary_chunk->maxlen - hashtable->primary_chunk->used) < size)
{
/* allocate new chunk and put it at the beginning of the list */
- newChunk = (HashMemoryChunk) MemoryContextAlloc(hashtable->batchCxt,
- offsetof(HashMemoryChunkData, data) + HASH_CHUNK_SIZE);
-
+ newChunk = (HashMemoryChunk)
+ MemoryContextAlloc(hashtable->batchCxt,
+ offsetof(HashMemoryChunkData, data) +
+ HASH_CHUNK_SIZE);
+ newChunk->next.private = hashtable->primary_chunk;
+ hashtable->primary_chunk = newChunk;
newChunk->maxlen = HASH_CHUNK_SIZE;
newChunk->used = size;
newChunk->ntuples = 1;
- newChunk->next = hashtable->chunks;
- hashtable->chunks = newChunk;
-
return newChunk->data;
}
/* There is enough space in the current chunk, let's add the tuple */
- ptr = hashtable->chunks->data + hashtable->chunks->used;
- hashtable->chunks->used += size;
- hashtable->chunks->ntuples += 1;
+ ptr = hashtable->primary_chunk->data + hashtable->primary_chunk->used;
+ hashtable->primary_chunk->used += size;
+ hashtable->primary_chunk->ntuples += 1;
/* return pointer to the start of the tuple memory */
return ptr;
diff --git a/src/backend/executor/nodeHashjoin.c b/src/backend/executor/nodeHashjoin.c
index 369e666..b8f90a6 100644
--- a/src/backend/executor/nodeHashjoin.c
+++ b/src/backend/executor/nodeHashjoin.c
@@ -21,8 +21,11 @@
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
#include "miscadmin.h"
+#include "pgstat.h"
+#include "storage/barrier.h"
#include "utils/memutils.h"
+#include <unistd.h> /* TODO: remove */
/*
* States of the ExecHashJoin state machine
@@ -42,11 +45,13 @@
static TupleTableSlot *ExecHashJoinOuterGetTuple(PlanState *outerNode,
HashJoinState *hjstate,
uint32 *hashvalue);
-static TupleTableSlot *ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
- BufFile *file,
+static TupleTableSlot *ExecHashJoinGetSavedTuple(HashJoinTable hashtable,
uint32 *hashvalue,
TupleTableSlot *tupleSlot);
static bool ExecHashJoinNewBatch(HashJoinState *hjstate);
+static void ExecHashJoinLoadBatch(HashJoinState *hjstate);
+static void ExecHashJoinExportBatches(HashJoinTable hashtable);
+static void ExecHashJoinPreloadNextBatch(HashJoinTable hashtable);
/* ----------------------------------------------------------------
@@ -147,6 +152,14 @@ ExecHashJoin(HashJoinState *node)
/* no chance to not build the hash table */
node->hj_FirstOuterTupleSlot = NULL;
}
+ else if (hashNode->shared_table_data != NULL)
+ {
+ /*
+ * TODO: The empty-outer optimization is not implemented
+ * for shared hash tables yet.
+ */
+ node->hj_FirstOuterTupleSlot = NULL;
+ }
else if (HJ_FILL_OUTER(node) ||
(outerNode->plan->startup_cost < hashNode->ps.plan->total_cost &&
!node->hj_OuterNotEmpty))
@@ -166,7 +179,7 @@ ExecHashJoin(HashJoinState *node)
/*
* create the hash table
*/
- hashtable = ExecHashTableCreate((Hash *) hashNode->ps.plan,
+ hashtable = ExecHashTableCreate(hashNode,
node->hj_HashOperators,
HJ_FILL_INNER(node));
node->hj_HashTable = hashtable;
@@ -177,12 +190,29 @@ ExecHashJoin(HashJoinState *node)
hashNode->hashtable = hashtable;
(void) MultiExecProcNode((PlanState *) hashNode);
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(&hashtable->shared->barrier) >=
+ PHJ_PHASE_HASHING);
+
+ /* Allow other backends to access batches we generated. */
+ ExecHashJoinExportBatches(hashtable);
+
+ /*
+ * Check if we are a worker that attached too late to
+ * avoid deadlock risk with the leader.
+ */
+ if (ExecHashCheckForEarlyExit(hashtable))
+ return NULL;
+ }
+
/*
* If the inner relation is completely empty, and we're not
* doing a left outer join, we can quit without scanning the
* outer relation.
*/
- if (hashtable->totalTuples == 0 && !HJ_FILL_OUTER(node))
+ if (!HashJoinTableIsShared(hashtable) && /* TODO:TM */
+ hashtable->totalTuples == 0 && !HJ_FILL_OUTER(node))
return NULL;
/*
@@ -198,12 +228,73 @@ ExecHashJoin(HashJoinState *node)
*/
node->hj_OuterNotEmpty = false;
- node->hj_JoinState = HJ_NEED_NEW_OUTER;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier = &hashtable->shared->barrier;
+ int phase = BarrierPhase(barrier);
+
+ /*
+ * Map the current phase to the appropriate initial state
+ * for this worker, so we can get started.
+ */
+ Assert(BarrierPhase(barrier) >= PHJ_PHASE_PROBING);
+ hashtable->curbatch = PHJ_PHASE_TO_BATCHNO(phase);
+ switch (PHJ_PHASE_TO_SUBPHASE(phase))
+ {
+ case PHJ_SUBPHASE_PROMOTING:
+ /* Wait for serial phase to finish. */
+ BarrierWait(barrier, WAIT_EVENT_HASHJOIN_PROMOTING);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(barrier)) ==
+ PHJ_SUBPHASE_LOADING);
+ /* fall through */
+ case PHJ_SUBPHASE_LOADING:
+ /* Help load the current batch. */
+ ExecHashUpdate(hashtable);
+ ExecHashJoinOpenBatch(hashtable, hashtable->curbatch,
+ true);
+ ExecHashJoinLoadBatch(node);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(barrier)) ==
+ PHJ_SUBPHASE_PROBING);
+ /* fall through */
+ case PHJ_SUBPHASE_PREPARING:
+ /* Wait for serial phase to finish. */
+ BarrierWait(barrier, WAIT_EVENT_HASHJOIN_PROMOTING);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(barrier)) ==
+ PHJ_SUBPHASE_PROBING);
+ /* fall through */
+ case PHJ_SUBPHASE_PROBING:
+ /* Help probe the current batch. */
+ ExecHashUpdate(hashtable);
+ ExecHashJoinOpenBatch(hashtable, hashtable->curbatch,
+ false);
+ node->hj_JoinState = HJ_NEED_NEW_OUTER;
+ break;
+ case PHJ_SUBPHASE_UNMATCHED:
+ /* Help scan for unmatched inner tuples. */
+ ExecHashUpdate(hashtable);
+ node->hj_JoinState = HJ_FILL_INNER_TUPLES;
+ break;
+ }
+ continue;
+ }
+ else
+ {
+ node->hj_JoinState = HJ_NEED_NEW_OUTER;
+ ExecHashJoinOpenBatch(hashtable, 0, false);
+ }
/* FALL THRU */
case HJ_NEED_NEW_OUTER:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(PHJ_PHASE_TO_BATCHNO(BarrierPhase(&hashtable->shared->barrier)) ==
+ hashtable->curbatch);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_PROBING);
+ }
+
/*
* We don't have an outer tuple, try to get the next one
*/
@@ -213,6 +304,47 @@ ExecHashJoin(HashJoinState *node)
if (TupIsNull(outerTupleSlot))
{
/* end of batch, or maybe whole join */
+
+ /*
+ * Switch to reading tuples from the next inner batch. We
+ * do this here because in the shared hash table case we
+ * want to do this before ExecHashJoinPreloadNextBatch.
+ */
+ if (hashtable->curbatch + 1 < hashtable->nbatch)
+ ExecHashJoinOpenBatch(hashtable,
+ hashtable->curbatch + 1,
+ true);
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Allow other backends to access our batches. */
+ ExecHashJoinExportBatches(hashtable);
+ /*
+ * Check if we are a leader that can't go further than
+ * probing the first batch without deadlock risk,
+ * because there are workers running.
+ */
+ if (ExecHashCheckForEarlyExit(hashtable))
+ return NULL;
+
+ /*
+ * We may be able to load some amount of the next
+ * batch into spare work_mem, before we start waiting
+ * for other workers to finish probing the current
+ * batch.
+ */
+ ExecHashJoinPreloadNextBatch(hashtable);
+
+ /*
+ * You can't start searching for unmatched tuples
+ * until all workers have finished probing, so we
+ * synchronize here.
+ */
+ BarrierWait(&hashtable->shared->barrier,
+ WAIT_EVENT_HASHJOIN_PROBING);
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_UNMATCHED_BATCH(hashtable->curbatch));
+ }
if (HJ_FILL_INNER(node))
{
/* set up to scan for unmatched inner tuples */
@@ -250,9 +382,9 @@ ExecHashJoin(HashJoinState *node)
* Save it in the corresponding outer-batch file.
*/
Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(ExecFetchSlotMinimalTuple(outerTupleSlot),
- hashvalue,
- &hashtable->outerBatchFile[batchno]);
+ ExecHashJoinSaveTuple(hashtable,
+ ExecFetchSlotMinimalTuple(outerTupleSlot),
+ hashvalue, batchno, false);
/* Loop around, staying in HJ_NEED_NEW_OUTER state */
continue;
}
@@ -296,6 +428,13 @@ ExecHashJoin(HashJoinState *node)
if (joinqual == NIL || ExecQual(joinqual, econtext, false))
{
node->hj_MatchedOuter = true;
+ /*
+ * Note: it is OK to do this in a shared hash table
+ * without any kind of memory synchronization, because the
+ * only transition is 0->1, so ordering doesn't matter if
+ * several backends do it, and there will be a memory
+ * barrier before anyone reads it.
+ */
HeapTupleHeaderSetMatch(HJTUPLE_MINTUPLE(node->hj_CurTuple));
/* In an antijoin, we never return a matched tuple */
@@ -632,6 +771,29 @@ ExecEndHashJoin(HashJoinState *node)
}
/*
+ * For shared hash joins, load as much of the next batch as we can as part of
+ * the probing phase for the current batch. This overlapping means that we do
+ * something useful before we start waiting for other workers.
+ */
+static void
+ExecHashJoinPreloadNextBatch(HashJoinTable hashtable)
+{
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier PG_USED_FOR_ASSERTS_ONLY = &hashtable->shared->barrier;
+ int curbatch = hashtable->curbatch;
+ int next_batch = curbatch + 1;
+
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_PROBING_BATCH(curbatch));
+
+ if (next_batch < hashtable->nbatch)
+ {
+ /* TODO: Load into secondary hash table while memory is free! */
+ }
+ }
+}
+
+/*
* ExecHashJoinOuterGetTuple
*
* get the next outer tuple for hashjoin: either by
@@ -702,8 +864,7 @@ ExecHashJoinOuterGetTuple(PlanState *outerNode,
if (file == NULL)
return NULL;
- slot = ExecHashJoinGetSavedTuple(hjstate,
- file,
+ slot = ExecHashJoinGetSavedTuple(hashtable,
hashvalue,
hjstate->hj_OuterTupleSlot);
if (!TupIsNull(slot))
@@ -726,13 +887,14 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
HashJoinTable hashtable = hjstate->hj_HashTable;
int nbatch;
int curbatch;
- BufFile *innerFile;
- TupleTableSlot *slot;
- uint32 hashvalue;
nbatch = hashtable->nbatch;
curbatch = hashtable->curbatch;
+ if (HashJoinTableIsShared(hashtable))
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_UNMATCHED_BATCH(curbatch));
+
if (curbatch > 0)
{
/*
@@ -776,7 +938,8 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
* need to be reassigned.
*/
curbatch++;
- while (curbatch < nbatch &&
+ while (!HashJoinTableIsShared(hashtable) &&
+ curbatch < nbatch &&
(hashtable->outerBatchFile[curbatch] == NULL ||
hashtable->innerBatchFile[curbatch] == NULL))
{
@@ -792,7 +955,6 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
if (hashtable->outerBatchFile[curbatch] &&
nbatch != hashtable->nbatch_outstart)
break; /* must process due to rule 3 */
- /* We can ignore this batch. */
/* Release associated temp files right away. */
if (hashtable->innerBatchFile[curbatch])
BufFileClose(hashtable->innerBatchFile[curbatch]);
@@ -812,48 +974,175 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
* Reload the hash table with the new inner batch (which could be empty)
*/
ExecHashTableReset(hashtable);
+ ExecHashJoinLoadBatch(hjstate);
- innerFile = hashtable->innerBatchFile[curbatch];
+ return true;
+}
+
+static void
+ExecHashJoinLoadBatch(HashJoinState *hjstate)
+{
+ HashJoinTable hashtable = hjstate->hj_HashTable;
+ int curbatch = hashtable->curbatch;
+ TupleTableSlot *slot;
+ uint32 hashvalue;
- if (innerFile != NULL)
+ if (HashJoinTableIsShared(hashtable))
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_LOADING);
+
+ /*
+ * In HJ_NEED_NEW_OUTER, we already selected the current inner batch for
+ * reading from. If there is a shared hash table, we may have already
+ * partially loaded the hash table in ExecHashJoinPreloadNextBatch.
+ */
+ Assert(hashtable->batch_reader.batchno = curbatch);
+ Assert(hashtable->batch_reader.inner);
+
+ for (;;)
{
- if (BufFileSeek(innerFile, 0, 0L, SEEK_SET))
- ereport(ERROR,
- (errcode_for_file_access(),
- errmsg("could not rewind hash-join temporary file: %m")));
+ slot = ExecHashJoinGetSavedTuple(hashtable,
+ &hashvalue,
+ hjstate->hj_HashTupleSlot);
- while ((slot = ExecHashJoinGetSavedTuple(hjstate,
- innerFile,
- &hashvalue,
- hjstate->hj_HashTupleSlot)))
- {
- /*
- * NOTE: some tuples may be sent to future batches. Also, it is
- * possible for hashtable->nbatch to be increased here!
- */
- ExecHashTableInsert(hashtable, slot, hashvalue);
- }
+ if (slot == NULL)
+ break;
/*
- * after we build the hash table, the inner batch file is no longer
- * needed
+ * NOTE: some tuples may be sent to future batches. Also, it is
+ * possible for hashtable->nbatch to be increased here!
*/
- BufFileClose(innerFile);
- hashtable->innerBatchFile[curbatch] = NULL;
+ ExecHashTableInsert(hashtable, slot, hashvalue, false);
}
/*
- * Rewind outer batch file (if present), so that we can start reading it.
+ * Now that we have finished loading this batch into the hash table, we
+ * can set our outer batch read head to the start of the current batch,
+ * and our inner batch read head to the start of the NEXT batch (as
+ * expected by ExecHashJoinPreloadNextBatch).
*/
- if (hashtable->outerBatchFile[curbatch] != NULL)
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Wait until all workers have finished loading their portion of the
+ * hash table.
+ */
+ if (BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASHJOIN_LOADING))
+ {
+ /* Serial phase: prepare to read this outer and next inner batch */
+ ExecHashJoinRewindBatches(hashtable, hashtable->curbatch);
+ }
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_PREPARING_BATCH(hashtable->curbatch));
+ BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASHJOIN_PREPARING);
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_PROBING_BATCH(hashtable->curbatch));
+ }
+ else
+ ExecHashJoinRewindBatches(hashtable, hashtable->curbatch);
+
+ /*
+ * The inner batch file is no longer needed by any participant, because
+ * the hash table has been fully reloaded.
+ */
+ ExecHashJoinCloseBatch(hashtable, hashtable->curbatch, true);
+
+ /* Prepare to read from the current outer batch. */
+ ExecHashJoinOpenBatch(hashtable, hashtable->curbatch, false);
+}
+
+/*
+ * Export a BufFile, copy the descriptor to DSA memory and return the
+ * dsa_pointer.
+ */
+static dsa_pointer
+make_batch_descriptor(dsa_area *area, BufFile *file)
+{
+ dsa_pointer pointer;
+ BufFileDescriptor *source;
+ BufFileDescriptor *target;
+ size_t size;
+
+ source = BufFileExport(file);
+ size = BufFileDescriptorSize(source);
+ pointer = dsa_allocate(area, size);
+ if (!DsaPointerIsValid(pointer))
+ ereport(ERROR,
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of memory"),
+ errdetail("Failed on dsa_allocate of size %zu.", size)));
+ target = dsa_get_address(area, pointer);
+ memcpy(target, source, size);
+ pfree(source);
+
+ return pointer;
+}
+
+/*
+ * Publish a batch descriptor for a future batch so that other participants
+ * can import it and read it. If 'descriptor' is InvalidDsaPointer, then
+ * forget the published descriptor so that it will be reexported later.
+ */
+static void
+set_batch_descriptor(HashJoinTable hashtable, int batchno, bool inner,
+ dsa_pointer descriptor)
+{
+ HashJoinParticipantState *participant;
+ dsa_pointer *level1;
+ dsa_pointer *level2;
+ int rank;
+ int index;
+
+ participant = &hashtable->shared->participants[HashJoinParticipantNumber()];
+ rank = fls(batchno);
+ index = batchno % (1 << (rank - 1));
+ level1 = inner ? participant->inner_batch_descriptors
+ : participant->outer_batch_descriptors;
+ if (level1[rank] == InvalidDsaPointer)
{
- if (BufFileSeek(hashtable->outerBatchFile[curbatch], 0, 0L, SEEK_SET))
+ size_t size = sizeof(dsa_pointer) * (1 << rank);
+
+ level1[rank] = dsa_allocate(hashtable->area, size);
+ if (level1[rank] == InvalidDsaPointer)
ereport(ERROR,
- (errcode_for_file_access(),
- errmsg("could not rewind hash-join temporary file: %m")));
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of memory"),
+ errdetail("Failed on dsa_allocate of size %zu.", size)));
+ level2 = dsa_get_address(hashtable->area, level1[rank]);
+ memset(level2, 0, size);
}
+ level2 = dsa_get_address(hashtable->area, level1[rank]);
+ if (level2[index] != InvalidDsaPointer)
+ dsa_free(hashtable->area, level2[index]);
+ level2[index] = descriptor;
+}
- return true;
+/*
+ * Get a batch descriptor published by a given participant, if there is one.
+ */
+static BufFileDescriptor *
+get_batch_descriptor(HashJoinTable hashtable, int participant_number,
+ int batchno, bool inner)
+{
+ HashJoinParticipantState *participant;
+ dsa_pointer *level1;
+ dsa_pointer *level2;
+ int rank;
+ int index;
+
+ participant = &hashtable->shared->participants[participant_number];
+ rank = fls(batchno);
+ index = batchno % (1 << (rank - 1));
+ level1 = inner ? participant->inner_batch_descriptors
+ : participant->outer_batch_descriptors;
+ if (level1[rank] == InvalidDsaPointer)
+ return NULL;
+ level2 = dsa_get_address(hashtable->area, level1[rank]);
+ if (level2[index] == InvalidDsaPointer)
+ return NULL;
+
+ return (BufFileDescriptor *)
+ dsa_get_address(hashtable->area, level2[index]);
}
/*
@@ -868,17 +1157,40 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
* will get messed up.
*/
void
-ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
- BufFile **fileptr)
+ExecHashJoinSaveTuple(HashJoinTable hashtable,
+ MinimalTuple tuple, uint32 hashvalue,
+ int batchno,
+ bool inner)
{
- BufFile *file = *fileptr;
+ BufFile *file;
size_t written;
+ if (inner)
+ file = hashtable->innerBatchFile[batchno];
+ else
+ file = hashtable->outerBatchFile[batchno];
if (file == NULL)
{
/* First write to this batch file, so open it. */
file = BufFileCreateTemp(false);
- *fileptr = file;
+ if (inner)
+ hashtable->innerBatchFile[batchno] = file;
+ else
+ hashtable->outerBatchFile[batchno] = file;
+ }
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* This batch needs to be re-exported, if it was already exported. */
+ /*
+ * TODO: This is far too expensive: need a bitmap? or maybe just
+ * export every batch when it's the next one to be processed,
+ * regardless of whether we've written anything to it (the point being
+ * that the list of files backing a BufFile can change when you write
+ * to it)? If we do that then we still need to export ALL before
+ * exiting early.
+ */
+ set_batch_descriptor(hashtable, batchno, inner, InvalidDsaPointer);
}
written = BufFileWrite(file, (void *) &hashvalue, sizeof(uint32));
@@ -895,54 +1207,337 @@ ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
}
/*
+ * Export unexported future batches created by this participant, so that other
+ * participants can read from them after they have finished reading their own.
+ */
+static void
+ExecHashJoinExportBatches(HashJoinTable hashtable)
+{
+ int i;
+
+ /* Find this participant's HashJoinParticipantState object. */
+ Assert(HashJoinParticipantNumber() < hashtable->shared->planned_participants);
+
+ /* Export future batches and copy their descriptors into DSA memory. */
+ for (i = hashtable->curbatch + 1; i < hashtable->nbatch; ++i)
+ {
+ if (hashtable->innerBatchFile[i] != NULL &&
+ get_batch_descriptor(hashtable, HashJoinParticipantNumber(), i, true) == InvalidDsaPointer)
+ set_batch_descriptor(hashtable, i, true,
+ make_batch_descriptor(hashtable->area, hashtable->innerBatchFile[i]));
+ if (hashtable->outerBatchFile[i] != NULL &&
+ get_batch_descriptor(hashtable, HashJoinParticipantNumber(), i, false) == InvalidDsaPointer)
+ set_batch_descriptor(hashtable, i, false,
+ make_batch_descriptor(hashtable->area, hashtable->outerBatchFile[i]));
+ }
+}
+
+/*
+ * Select the batch file that ExecHashJoinGetSavedTuple will read from.
+ */
+void
+ExecHashJoinOpenBatch(HashJoinTable hashtable, int batchno, bool inner)
+{
+ HashJoinBatchReader *batch_reader = &hashtable->batch_reader;
+
+ if (batchno == 0)
+ batch_reader->file = NULL;
+ else
+ batch_reader->file = inner
+ ? hashtable->innerBatchFile[batchno]
+ : hashtable->outerBatchFile[batchno];
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ HashJoinParticipantState *participant;
+
+ participant =
+ &hashtable->shared->participants[HashJoinParticipantNumber()];
+ batch_reader->shared = inner
+ ? &participant->inner_batch_reader
+ : &participant->outer_batch_reader;
+ /* We will seek to the shared position at next read. */
+ batch_reader->head.fileno = -1;
+ batch_reader->head.offset = -1;
+ }
+ else
+ {
+ batch_reader->shared = NULL;
+ /* Seek to start of batch now, if there is one. */
+ if (batch_reader->file != NULL)
+ BufFileSeek(batch_reader->file, 0, 0, SEEK_SET);
+ }
+
+ batch_reader->participant_number = HashJoinParticipantNumber();
+ batch_reader->batchno = batchno;
+ batch_reader->inner = inner;
+
+}
+
+/*
+ * Close a batch, once it is not needed by any participant. This causes batch
+ * files created by this participant to be deleted.
+ */
+void
+ExecHashJoinCloseBatch(HashJoinTable hashtable, int batchno, bool inner)
+{
+ HashJoinParticipantState *participant;
+ HashJoinBatchReader *batch_reader;
+ BufFile *file;
+
+ /*
+ * We only need to close the batch owned by THIS participant. That causes
+ * it to be deleted. Batches opened in this backend but created by other
+ * participants are closed by ExecHashJoinGetSavedTuple when it reaches
+ * the end of the file, allowing them to be closed sooner.
+ */
+ batch_reader = &hashtable->batch_reader;
+ participant = &hashtable->shared->participants[HashJoinParticipantNumber()];
+ if (inner)
+ {
+ file = hashtable->innerBatchFile[batchno];
+ hashtable->innerBatchFile[batchno] = NULL;
+ }
+ else
+ {
+ file = hashtable->outerBatchFile[batchno];
+ hashtable->outerBatchFile[batchno] = NULL;
+ }
+ if (file == NULL)
+ return;
+
+ Assert(batch_reader->file == NULL || file == batch_reader->file);
+ BufFileClose(file);
+ batch_reader->file = NULL;
+}
+
+/*
+ * Rewind batch readers. The outer batch reader is rewound to the start of
+ * batchno. The inner batch reader is rewound to the start of batchno + 1, in
+ * anticipation of preloading the next batch.
+ */
+void
+ExecHashJoinRewindBatches(HashJoinTable hashtable, int batchno)
+{
+ HashJoinBatchReader *batch_reader;
+ int i;
+
+ batch_reader = &hashtable->batch_reader;
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(&hashtable->shared->barrier) == PHJ_PHASE_CREATING ||
+ (PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_PREPARING &&
+ PHJ_PHASE_TO_BATCHNO(BarrierPhase(&hashtable->shared->barrier)) ==
+ batchno));
+
+ /*
+ * Position the shared read heads for each participant's batch.
+ * Readers will seek their BufFile as required to synchronize.
+ */
+ for (i = 0; i < hashtable->shared->planned_participants; ++i)
+ {
+ HashJoinSharedBatchReader *reader;
+
+ reader = &hashtable->shared->participants[i].outer_batch_reader;
+ reader->batchno = batchno; /* for probing this batch */
+ reader->head.fileno = 0;
+ reader->head.offset = 0;
+
+ reader = &hashtable->shared->participants[i].inner_batch_reader;
+ reader->batchno = batchno + 1; /* for preloading the next batch */
+ reader->head.fileno = 0;
+ reader->head.offset = 0;
+ }
+ }
+}
+
+/*
* ExecHashJoinGetSavedTuple
- * read the next tuple from a batch file. Return NULL if no more.
+ * read the next tuple from the batch selected with
+ * ExecHashJoinOpenBatch, including the batch files of
+ * other participants if the hash table is shared. Return NULL if no
+ * more.
*
* On success, *hashvalue is set to the tuple's hash value, and the tuple
* itself is stored in the given slot.
*/
static TupleTableSlot *
-ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
- BufFile *file,
+ExecHashJoinGetSavedTuple(HashJoinTable hashtable,
uint32 *hashvalue,
TupleTableSlot *tupleSlot)
{
- uint32 header[2];
- size_t nread;
- MinimalTuple tuple;
+ TupleTableSlot *result = NULL;
+ HashJoinBatchReader *batch_reader = &hashtable->batch_reader;
+ BufFileDescriptor *descriptor;
- /*
- * Since both the hash value and the MinimalTuple length word are uint32,
- * we can read them both in one BufFileRead() call without any type
- * cheating.
- */
- nread = BufFileRead(file, (void *) header, sizeof(header));
- if (nread == 0) /* end of file */
+ for (;;)
{
- ExecClearTuple(tupleSlot);
- return NULL;
- }
- if (nread != sizeof(header))
- ereport(ERROR,
- (errcode_for_file_access(),
+ uint32 header[2];
+ size_t nread;
+ MinimalTuple tuple;
+ bool can_close = false;
+
+ if (batch_reader->file == NULL)
+ {
+ /*
+ * No file found for the current participant. Try stealing tuples
+ * from the next participant.
+ */
+ goto next_participant;
+ }
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ LWLockAcquire(&batch_reader->shared->lock, LW_EXCLUSIVE);
+ Assert(batch_reader->shared->batchno == batch_reader->batchno);
+ if (batch_reader->shared->error)
+ {
+ /* Don't try to read if reading failed in some other backend. */
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from hash-join temporary file")));
+ }
+
+ /* Set the shared error flag, which we'll clear if we succeed. */
+ batch_reader->shared->error = true;
+
+ /*
+ * If another worker has moved the shared read head since we last read,
+ * we'll need to seek to the new shared position.
+ */
+ if (batch_reader->head.fileno != batch_reader->shared->head.fileno ||
+ batch_reader->head.offset != batch_reader->shared->head.offset)
+ {
+ BufFileSeek(batch_reader->file,
+ batch_reader->shared->head.fileno,
+ batch_reader->shared->head.offset,
+ SEEK_SET);
+ batch_reader->head = batch_reader->shared->head;
+ }
+ }
+
+ /* Try to read the size and hash. */
+ nread = BufFileRead(batch_reader->file, (void *) header, sizeof(header));
+ if (nread > 0)
+ {
+ if (nread != sizeof(header))
+ {
+ ereport(ERROR,
+ (errcode_for_file_access(),
errmsg("could not read from hash-join temporary file: %m")));
- *hashvalue = header[0];
- tuple = (MinimalTuple) palloc(header[1]);
- tuple->t_len = header[1];
- nread = BufFileRead(file,
- (void *) ((char *) tuple + sizeof(uint32)),
- header[1] - sizeof(uint32));
- if (nread != header[1] - sizeof(uint32))
- ereport(ERROR,
- (errcode_for_file_access(),
+ }
+ *hashvalue = header[0];
+ tuple = (MinimalTuple) palloc(header[1]);
+ tuple->t_len = header[1];
+ nread = BufFileRead(batch_reader->file,
+ (void *) ((char *) tuple + sizeof(uint32)),
+ header[1] - sizeof(uint32));
+ if (nread != header[1] - sizeof(uint32))
+ {
+ ereport(ERROR,
+ (errcode_for_file_access(),
errmsg("could not read from hash-join temporary file: %m")));
- return ExecStoreMinimalTuple(tuple, tupleSlot, true);
-}
+ }
+
+ result = ExecStoreMinimalTuple(tuple, tupleSlot, true);
+
+ }
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ if (nread == 0 &&
+ batch_reader->participant_number !=
+ HashJoinParticipantNumber())
+ {
+ /*
+ * We've reached the end of another paticipant's batch file,
+ * so close it now. We'll deal with closing THIS
+ * participant's batch file later, because we don't want the
+ * files to be deleted just yet.
+ */
+ can_close = true;
+ }
+ /* Commit new head position to shared memory and clear error. */
+ BufFileTell(batch_reader->file,
+ &batch_reader->head.fileno,
+ &batch_reader->head.offset);
+ batch_reader->shared->head = batch_reader->head;
+ batch_reader->shared->error = false;
+ LWLockRelease(&batch_reader->shared->lock);
+ }
+
+ if (can_close)
+ {
+ BufFileClose(batch_reader->file);
+ batch_reader->file = NULL;
+ }
+
+ if (result != NULL)
+ return result;
+
+next_participant:
+ if (!HashJoinTableIsShared(hashtable))
+ {
+ /* Private hash table, end of batch. */
+ ExecClearTuple(tupleSlot);
+ return NULL;
+ }
+
+ /* Try the next participant's batch file. */
+ batch_reader->participant_number =
+ (batch_reader->participant_number + 1) %
+ hashtable->shared->planned_participants;
+ if (batch_reader->participant_number == HashJoinParticipantNumber())
+ {
+ /*
+ * We've made it all the way back to the file we started with,
+ * which is the one that this backend wrote. So there are no more
+ * tuples to be had in any participant's batch file.
+ */
+ ExecClearTuple(tupleSlot);
+ return NULL;
+ }
+ /* Import the BufFile from that participant, if it exported one. */
+ descriptor = get_batch_descriptor(hashtable,
+ batch_reader->participant_number,
+ batch_reader->batchno,
+ batch_reader->inner);
+ if (descriptor == NULL)
+ batch_reader->file = NULL;
+ else
+ {
+ HashJoinParticipantState *participant;
+
+ batch_reader->file = BufFileImport(descriptor);
+ participant =
+ &hashtable->shared->participants[batch_reader->participant_number];
+ if (batch_reader->inner)
+ batch_reader->shared = &participant->inner_batch_reader;
+ else
+ batch_reader->shared = &participant->outer_batch_reader;
+ batch_reader->head.fileno = batch_reader->head.offset = -1;
+ }
+ }
+}
void
ExecReScanHashJoin(HashJoinState *node)
{
+ HashState *hashNode = (HashState *) innerPlanState(node);
+
+ /* We can't use HashJoinTableIsShared if the table is NULL. */
+ if (hashNode->shared_table_data != NULL)
+ {
+ elog(ERROR, "TODO: shared ExecReScanHashJoin not implemented");
+
+ /* Coordinate a rewind to the shared hash table creation phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier, PHJ_PHASE_INIT,
+ WAIT_EVENT_HASHJOIN_REWINDING);
+ }
+
/*
* In a multi-batch join, we currently have to do rescans the hard way,
* primarily because batch temp files may have already been released. But
@@ -977,6 +1572,14 @@ ExecReScanHashJoin(HashJoinState *node)
/* ExecHashJoin can skip the BUILD_HASHTABLE step */
node->hj_JoinState = HJ_NEED_NEW_OUTER;
+
+ if (HashJoinTableIsShared(node->hj_HashTable))
+ {
+ /* Coordinate a rewind to the shared probing phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier,
+ PHJ_PHASE_PROBING,
+ WAIT_EVENT_HASHJOIN_REWINDING2);
+ }
}
else
{
@@ -985,6 +1588,14 @@ ExecReScanHashJoin(HashJoinState *node)
node->hj_HashTable = NULL;
node->hj_JoinState = HJ_BUILD_HASHTABLE;
+ if (HashJoinTableIsShared(node->hj_HashTable))
+ {
+ /* Coordinate a rewind to the shared hash table creation phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier,
+ PHJ_PHASE_INIT,
+ WAIT_EVENT_HASHJOIN_REWINDING3);
+ }
+
/*
* if chgParam of subnode is not null then plan will be re-scanned
* by first ExecProcNode.
@@ -1011,3 +1622,110 @@ ExecReScanHashJoin(HashJoinState *node)
if (node->js.ps.lefttree->chgParam == NULL)
ExecReScan(node->js.ps.lefttree);
}
+
+void ExecHashJoinEstimate(HashJoinState *state, ParallelContext *pcxt)
+{
+ size_t size;
+
+ size = offsetof(SharedHashJoinTableData, participants) +
+ sizeof(HashJoinParticipantState) * (pcxt->nworkers + 1);
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+static void
+configure_reader_locks(HashJoinParticipantState *participants, int count)
+{
+ int i;
+
+ static LWLockTranche inner_tranche;
+ static LWLockTranche outer_tranche;
+
+ inner_tranche.name = "Hash Join/inner batch";
+ inner_tranche.array_base =
+ (char *) &participants[0].inner_batch_reader.lock;
+ inner_tranche.array_stride = sizeof(HashJoinParticipantState);
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_HASH_JOIN_INNER_BATCH_READER,
+ &inner_tranche);
+
+ outer_tranche.name = "Hash Join/outer batch";
+ outer_tranche.array_base =
+ (char *) &participants[0].outer_batch_reader.lock;
+ outer_tranche.array_stride = sizeof(HashJoinParticipantState);
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_HASH_JOIN_OUTER_BATCH_READER,
+ &outer_tranche);
+
+ for (i = 0; i < count; ++i)
+ {
+ LWLockInitialize(&participants[i].inner_batch_reader.lock,
+ LWTRANCHE_PARALLEL_HASH_JOIN_INNER_BATCH_READER);
+ LWLockInitialize(&participants[i].outer_batch_reader.lock,
+ LWTRANCHE_PARALLEL_HASH_JOIN_OUTER_BATCH_READER);
+ }
+}
+
+void
+ExecHashJoinInitializeDSM(HashJoinState *state, ParallelContext *pcxt)
+{
+ HashState *hashNode;
+ SharedHashJoinTable shared;
+ size_t size;
+ int planned_participants;
+
+ /*
+ * Set up the state needed to coordinate access to the shared hash table,
+ * using the plan node ID as the toc key.
+ */
+ planned_participants = pcxt->nworkers + 1; /* possible workers + leader */
+ size = offsetof(SharedHashJoinTableData, participants) +
+ sizeof(HashJoinParticipantState) * planned_participants;
+ shared = shm_toc_allocate(pcxt->toc, size);
+ BarrierInit(&shared->barrier, 0);
+ shared->primary_buckets = InvalidDsaPointer;
+ shared->secondary_buckets = InvalidDsaPointer;
+ pg_atomic_init_u32(&shared->next_unmatched_bucket, 0);
+ pg_atomic_init_u64(&shared->total_primary_tuples, 0);
+ pg_atomic_init_u64(&shared->total_secondary_tuples, 0);
+ dsa_pointer_atomic_init(&shared->head_primary_chunk, InvalidDsaPointer);
+ dsa_pointer_atomic_init(&shared->head_secondary_chunk, InvalidDsaPointer);
+ dsa_pointer_atomic_init(&shared->chunks_to_rebucket, InvalidDsaPointer);
+ shared->planned_participants = planned_participants;
+ shm_toc_insert(pcxt->toc, state->js.ps.plan->plan_node_id, shared);
+ configure_reader_locks(shared->participants, planned_participants);
+
+ /*
+ * Pass the SharedHashJoinTable to the hash node. If the Gather node
+ * running in the leader backend decides to execute the hash join, it
+ * hasn't called ExecHashJoinInitializeWorker so it doesn't have
+ * state->shared_table_data set up. So we must do it here.
+ */
+ hashNode = (HashState *) innerPlanState(state);
+ hashNode->shared_table_data = shared;
+}
+
+void
+ExecHashJoinInitializeWorker(HashJoinState *state, shm_toc *toc)
+{
+ HashState *hashNode;
+
+ state->hj_sharedHashJoinTable =
+ shm_toc_lookup(toc, state->js.ps.plan->plan_node_id);
+
+ /*
+ * Inject SharedHashJoinTable into the hash node. It could instead have
+ * its own ExecHashInitializeWorker function, but we only want to set its
+ * 'parallel_aware' flag if we want to tell it to actually build the hash
+ * table in parallel. Since its parallel_aware flag also controls whether
+ * its 'InitializeWorker' function gets called, and it also needs access
+ * to this object for serial shared hash mode, we'll pass it on here
+ * instead of depending on that.
+ */
+ hashNode = (HashState *) innerPlanState(state);
+ hashNode->shared_table_data = state->hj_sharedHashJoinTable;
+ Assert(hashNode->shared_table_data != NULL);
+
+ Assert(HashJoinParticipantNumber() <
+ hashNode->shared_table_data->planned_participants);
+
+ configure_reader_locks(hashNode->shared_table_data->participants, 0);
+}
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 00bf3a5..361eb5d 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -31,6 +31,8 @@
#include "executor/nodeSeqscan.h"
#include "utils/rel.h"
+#include <unistd.h>
+
static void InitScanRelation(SeqScanState *node, EState *estate, int eflags);
static TupleTableSlot *SeqNext(SeqScanState *node);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index ae86954..ca215dd 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1993,6 +1993,7 @@ _outHashPath(StringInfo str, const HashPath *node)
WRITE_NODE_FIELD(path_hashclauses);
WRITE_INT_FIELD(num_batches);
+ WRITE_ENUM_FIELD(table_type, HashPathTableType);
}
static void
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 2a49639..79c7650 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -104,6 +104,7 @@
double seq_page_cost = DEFAULT_SEQ_PAGE_COST;
double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
+double cpu_shared_tuple_cost = DEFAULT_CPU_SHARED_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
double parallel_tuple_cost = DEFAULT_PARALLEL_TUPLE_COST;
@@ -2694,7 +2695,8 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
List *hashclauses,
Path *outer_path, Path *inner_path,
SpecialJoinInfo *sjinfo,
- SemiAntiJoinFactors *semifactors)
+ SemiAntiJoinFactors *semifactors,
+ HashPathTableType table_type)
{
Cost startup_cost = 0;
Cost run_cost = 0;
@@ -2725,6 +2727,26 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
run_cost += cpu_operator_cost * num_hashclauses * outer_path_rows;
/*
+ * If this is a shared hash table, there is a extra charge for inserting
+ * each tuple into the shared hash table, to cover the overhead of memory
+ * synchronization that makes the hash table slightly slower to build than
+ * a private hash table. There is no extra charge for probing the hash
+ * table for outer path row, on the basis that read-only access to the
+ * hash table shouldn't generate any extra memory synchronization.
+ *
+ * TODO: Really what we want is some guess at the number of cache sync
+ * overhead generated by inserting into cachelines that have been
+ * invalidated by someone else inserting into a bucket in the same
+ * cacheline. Not sure if it's better to introduce a
+ * cpu_cacheline_sync_cost (or _miss_cost?) and then here estimate the
+ * number of collisions we expect based by num buckets, cacheline size,
+ * num workers. But that might be too detailed/low level/variable
+ * heavy/bogus.
+ */
+ if (table_type != HASHPATH_TABLE_PRIVATE)
+ startup_cost += cpu_shared_tuple_cost * inner_path_rows;
+
+ /*
* Get hash table size that executor would use for inner relation.
*
* XXX for the moment, always assume that skew optimization will be
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index cc7384f..87c4cef 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -483,7 +483,8 @@ try_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *hashclauses,
JoinType jointype,
- JoinPathExtraData *extra)
+ JoinPathExtraData *extra,
+ HashPathTableType table_type)
{
Relids required_outer;
JoinCostWorkspace workspace;
@@ -508,7 +509,7 @@ try_hashjoin_path(PlannerInfo *root,
*/
initial_cost_hashjoin(root, &workspace, jointype, hashclauses,
outer_path, inner_path,
- extra->sjinfo, &extra->semifactors);
+ extra->sjinfo, &extra->semifactors, table_type);
if (add_path_precheck(joinrel,
workspace.startup_cost, workspace.total_cost,
@@ -525,7 +526,8 @@ try_hashjoin_path(PlannerInfo *root,
inner_path,
extra->restrictlist,
required_outer,
- hashclauses));
+ hashclauses,
+ table_type));
}
else
{
@@ -546,7 +548,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *hashclauses,
JoinType jointype,
- JoinPathExtraData *extra)
+ JoinPathExtraData *extra,
+ HashPathTableType table_type)
{
JoinCostWorkspace workspace;
@@ -571,7 +574,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
*/
initial_cost_hashjoin(root, &workspace, jointype, hashclauses,
outer_path, inner_path,
- extra->sjinfo, &extra->semifactors);
+ extra->sjinfo, &extra->semifactors,
+ table_type);
if (!add_partial_path_precheck(joinrel, workspace.total_cost, NIL))
return;
@@ -587,7 +591,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
inner_path,
extra->restrictlist,
NULL,
- hashclauses));
+ hashclauses,
+ table_type));
}
/*
@@ -1356,7 +1361,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
/* no possibility of cheap startup here */
}
else if (jointype == JOIN_UNIQUE_INNER)
@@ -1372,7 +1378,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
if (cheapest_startup_outer != NULL &&
cheapest_startup_outer != cheapest_total_outer)
try_hashjoin_path(root,
@@ -1381,7 +1388,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
}
else
{
@@ -1402,7 +1410,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
foreach(lc1, outerrel->cheapest_parameterized_paths)
{
@@ -1436,7 +1445,8 @@ hash_inner_and_outer(PlannerInfo *root,
innerpath,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
}
}
}
@@ -1445,23 +1455,32 @@ hash_inner_and_outer(PlannerInfo *root,
* If the joinrel is parallel-safe, we may be able to consider a
* partial hash join. However, we can't handle JOIN_UNIQUE_OUTER,
* because the outer path will be partial, and therefore we won't be
- * able to properly guarantee uniqueness. Similarly, we can't handle
- * JOIN_FULL and JOIN_RIGHT, because they can produce false null
- * extended rows. Also, the resulting path must not be parameterized.
+ * able to properly guarantee uniqueness. Also, the resulting path
+ * must not be parameterized.
*/
if (joinrel->consider_parallel &&
jointype != JOIN_UNIQUE_OUTER &&
- jointype != JOIN_FULL &&
- jointype != JOIN_RIGHT &&
outerrel->partial_pathlist != NIL &&
bms_is_empty(joinrel->lateral_relids))
{
Path *cheapest_partial_outer;
+ 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? */
+ if (innerrel->partial_pathlist != NIL)
+ cheapest_partial_inner =
+ (Path *) linitial(innerrel->partial_pathlist);
+ if (cheapest_partial_inner != NULL)
+ try_partial_hashjoin_path(root, joinrel,
+ cheapest_partial_outer,
+ cheapest_partial_inner,
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_SHARED_PARALLEL);
+
/*
* Normally, given that the joinrel is parallel-safe, the cheapest
* total inner path will also be parallel-safe, but if not, we'll
@@ -1488,10 +1507,20 @@ hash_inner_and_outer(PlannerInfo *root,
}
if (cheapest_safe_inner != NULL)
+ {
+ /* Try a shared table with only one worker building the table. */
try_partial_hashjoin_path(root, joinrel,
cheapest_partial_outer,
cheapest_safe_inner,
- hashclauses, jointype, extra);
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_SHARED_SERIAL);
+ /* Also private hash tables, built by each worker. */
+ try_partial_hashjoin_path(root, joinrel,
+ cheapest_partial_outer,
+ cheapest_safe_inner,
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_PRIVATE);
+ }
}
}
}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index ad49674..4954c4c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -3938,6 +3938,23 @@ create_hashjoin_plan(PlannerInfo *root,
copy_plan_costsize(&hash_plan->plan, inner_plan);
hash_plan->plan.startup_cost = hash_plan->plan.total_cost;
+ /*
+ * Set the table as sharable if appropriate, with parallel or serial
+ * building.
+ */
+ switch (best_path->table_type)
+ {
+ case HASHPATH_TABLE_SHARED_PARALLEL:
+ hash_plan->shared_table = true;
+ hash_plan->plan.parallel_aware = true;
+ break;
+ case HASHPATH_TABLE_SHARED_SERIAL:
+ hash_plan->shared_table = true;
+ break;
+ case HASHPATH_TABLE_PRIVATE:
+ break;
+ }
+
join_plan = make_hashjoin(tlist,
joinclauses,
otherclauses,
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index abb7507..68cabe6 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -2096,6 +2096,7 @@ create_mergejoin_path(PlannerInfo *root,
* 'required_outer' is the set of required outer rels
* 'hashclauses' are the RestrictInfo nodes to use as hash clauses
* (this should be a subset of the restrict_clauses list)
+ * 'table_type' for level of hash table sharing
*/
HashPath *
create_hashjoin_path(PlannerInfo *root,
@@ -2108,7 +2109,8 @@ create_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *restrict_clauses,
Relids required_outer,
- List *hashclauses)
+ List *hashclauses,
+ HashPathTableType table_type)
{
HashPath *pathnode = makeNode(HashPath);
@@ -2123,9 +2125,13 @@ create_hashjoin_path(PlannerInfo *root,
sjinfo,
required_outer,
&restrict_clauses);
- pathnode->jpath.path.parallel_aware = false;
+ pathnode->jpath.path.parallel_aware =
+ joinrel->consider_parallel &&
+ (table_type == HASHPATH_TABLE_SHARED_SERIAL ||
+ table_type == HASHPATH_TABLE_SHARED_PARALLEL);
pathnode->jpath.path.parallel_safe = joinrel->consider_parallel &&
outer_path->parallel_safe && inner_path->parallel_safe;
+ pathnode->table_type = table_type;
/* This is a foolish way to estimate parallel_workers, but for now... */
pathnode->jpath.path.parallel_workers = outer_path->parallel_workers;
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index a392197..c1e8819 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3393,6 +3393,51 @@ pgstat_get_wait_ipc(WaitEventIPC w)
case WAIT_EVENT_SYNC_REP:
event_name = "SyncRep";
break;
+ case WAIT_EVENT_HASH_CREATING:
+ event_name = "Hash/Creating";
+ break;
+ case WAIT_EVENT_HASH_HASHING:
+ event_name = "Hash/Hashing";
+ break;
+ case WAIT_EVENT_HASH_RESIZING:
+ event_name = "Hash/Resizing";
+ break;
+ case WAIT_EVENT_HASH_REBUCKETING:
+ event_name = "Hash/Rebucketing";
+ break;
+ case WAIT_EVENT_HASH_INIT:
+ event_name = "Hash/Init";
+ break;
+ case WAIT_EVENT_HASH_DESTROY:
+ event_name = "Hash/Destroy";
+ break;
+ case WAIT_EVENT_HASH_UNMATCHED:
+ event_name = "Hash/Unmatched";
+ break;
+ case WAIT_EVENT_HASH_PROMOTING:
+ event_name = "Hash/Promoting";
+ break;
+ case WAIT_EVENT_HASHJOIN_PROMOTING:
+ event_name = "HashJoin/Promoting";
+ break;
+ case WAIT_EVENT_HASHJOIN_PREPARING:
+ event_name = "HashJoin/Preparing";
+ break;
+ case WAIT_EVENT_HASHJOIN_PROBING:
+ event_name = "HashJoin/Probing";
+ break;
+ case WAIT_EVENT_HASHJOIN_LOADING:
+ event_name = "HashJoin/Loading";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING:
+ event_name = "HashJoin/Rewinding";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING2:
+ event_name = "HashJoin/Rewinding2";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING3:
+ event_name = "HashJoin/Rewinding3";;
+ break;
/* no default case, so that compiler will warn */
}
diff --git a/src/backend/storage/file/buffile.c b/src/backend/storage/file/buffile.c
index 042be79..0fc8404 100644
--- a/src/backend/storage/file/buffile.c
+++ b/src/backend/storage/file/buffile.c
@@ -42,6 +42,8 @@
#include "storage/buf_internals.h"
#include "utils/resowner.h"
+extern int ParallelWorkerNumber;
+
/*
* We break BufFiles into gigabyte-sized segments, regardless of RELSEG_SIZE.
* The reason is that we'd like large temporary BufFiles to be spread across
@@ -89,6 +91,24 @@ struct BufFile
char buffer[BLCKSZ];
};
+/*
+ * Serialized representation of a single file managed by a BufFile.
+ */
+typedef struct BufFileFileDescriptor
+{
+ char path[MAXPGPATH];
+} BufFileFileDescriptor;
+
+/*
+ * Serialized representation of a BufFile, to be created by BufFileExport and
+ * consumed by BufFileImport.
+ */
+struct BufFileDescriptor
+{
+ size_t num_files;
+ BufFileFileDescriptor files[FLEXIBLE_ARRAY_MEMBER];
+};
+
static BufFile *makeBufFile(File firstfile);
static void extendBufFile(BufFile *file);
static void BufFileLoadBuffer(BufFile *file);
@@ -178,6 +198,77 @@ BufFileCreateTemp(bool interXact)
return file;
}
+/*
+ * Export a BufFile description in a serialized form so that another backend
+ * can attach to it and read from it. The format is opaque, but it may be
+ * bitwise copied, and its size may be obtained with BufFileDescriptorSize().
+ */
+BufFileDescriptor *
+BufFileExport(BufFile *file)
+{
+ BufFileDescriptor *descriptor;
+ int i;
+
+ /* Flush output from local buffers. */
+ BufFileFlush(file);
+
+ /* Create and fill in a descriptor. */
+ descriptor = palloc0(offsetof(BufFileDescriptor, files) +
+ sizeof(BufFileFileDescriptor) * file->numFiles);
+ descriptor->num_files = file->numFiles;
+ for (i = 0; i < descriptor->num_files; ++i)
+ strcpy(descriptor->files[i].path, FilePathName(file->files[i]));
+
+ return descriptor;
+}
+
+/*
+ * Return the size in bytes of a BufFileDescriptor, so that it can be copied.
+ */
+size_t
+BufFileDescriptorSize(const BufFileDescriptor *descriptor)
+{
+ return offsetof(BufFileDescriptor, files) +
+ sizeof(BufFileFileDescriptor) * descriptor->num_files;
+}
+
+/*
+ * Open a BufFile that was created by another backend and then exported. The
+ * file must be read-only in all backends, and is still owned by the backend
+ * that created it. This provides a way for cooperating backends to share
+ * immutable temporary data such as hash join batches.
+ */
+BufFile *
+BufFileImport(BufFileDescriptor *descriptor)
+{
+ BufFile *file = (BufFile *) palloc(sizeof(BufFile));
+ int i;
+
+ file->numFiles = descriptor->num_files;
+ file->files = (File *) palloc0(sizeof(File) * descriptor->num_files);
+ file->offsets = (off_t *) palloc0(sizeof(off_t) * descriptor->num_files);
+ file->isTemp = false;
+ file->isInterXact = true; /* prevent cleanup by this backend */
+ file->dirty = false;
+ file->resowner = CurrentResourceOwner;
+ file->curFile = 0;
+ file->curOffset = 0L;
+ file->pos = 0;
+ file->nbytes = 0;
+
+ for (i = 0; i < descriptor->num_files; ++i)
+ {
+ file->files[i] =
+ PathNameOpenFile(descriptor->files[i].path,
+ O_RDONLY | PG_BINARY, 0600);
+ if (file->files[i] <= 0)
+ elog(ERROR, "failed to import \"%s\": %m",
+ descriptor->files[i].path);
+ }
+
+ return file;
+}
+
#ifdef NOT_USED
/*
* Create a BufFile and attach it to an already-opened virtual File.
diff --git a/src/backend/storage/ipc/barrier.c b/src/backend/storage/ipc/barrier.c
index 8b83c1d..5a45103 100644
--- a/src/backend/storage/ipc/barrier.c
+++ b/src/backend/storage/ipc/barrier.c
@@ -16,6 +16,7 @@
#include "storage/barrier.h"
+
/*
* Initialize this barrier, setting a static number of participants that we
* will wait for at each computation phase. To use a dynamic number of
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 2d3cf9e..6c79733 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -788,7 +788,6 @@ pg_stat_get_activity(PG_FUNCTION_ARGS)
raw_wait_event = UINT32_ACCESS_ONCE(proc->wait_event_info);
wait_event_type = pgstat_get_wait_event_type(raw_wait_event);
wait_event = pgstat_get_wait_event(raw_wait_event);
-
}
else
{
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 65660c1..9b49918 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2857,6 +2857,16 @@ static struct config_real ConfigureNamesReal[] =
NULL, NULL, NULL
},
{
+ {"cpu_shared_tuple_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "sharing each tuple with other parallel workers."),
+ NULL
+ },
+ &cpu_shared_tuple_cost,
+ DEFAULT_CPU_TUPLE_COST, -DBL_MAX, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
{"cpu_index_tuple_cost", PGC_USERSET, QUERY_TUNING_COST,
gettext_noop("Sets the planner's estimate of the cost of "
"processing each index entry during an index scan."),
diff --git a/src/include/executor/hashjoin.h b/src/include/executor/hashjoin.h
index 6d0e12b..1bbf376 100644
--- a/src/include/executor/hashjoin.h
+++ b/src/include/executor/hashjoin.h
@@ -15,7 +15,13 @@
#define HASHJOIN_H
#include "nodes/execnodes.h"
+#include "port/atomics.h"
+#include "storage/barrier.h"
#include "storage/buffile.h"
+#include "storage/dsa.h"
+#include "storage/fd.h"
+#include "storage/lwlock.h"
+#include "storage/spin.h"
/* ----------------------------------------------------------------
* hash-join hash table structures
@@ -63,7 +69,12 @@
typedef struct HashJoinTupleData
{
- struct HashJoinTupleData *next; /* link to next tuple in same bucket */
+ /* link to next tuple in same bucket */
+ union
+ {
+ dsa_pointer shared;
+ struct HashJoinTupleData *private;
+ } next;
uint32 hashvalue; /* tuple's hash code */
/* Tuple data, in MinimalTuple format, follows on a MAXALIGN boundary */
} HashJoinTupleData;
@@ -94,7 +105,12 @@ typedef struct HashJoinTupleData
typedef struct HashSkewBucket
{
uint32 hashvalue; /* common hash value */
- HashJoinTuple tuples; /* linked list of inner-relation tuples */
+ /* linked list of inner-relation tuples */
+ union
+ {
+ dsa_pointer shared;
+ HashJoinTuple private;
+ } tuples;
} HashSkewBucket;
#define SKEW_BUCKET_OVERHEAD MAXALIGN(sizeof(HashSkewBucket))
@@ -103,8 +119,9 @@ typedef struct HashSkewBucket
#define SKEW_MIN_OUTER_FRACTION 0.01
/*
- * To reduce palloc overhead, the HashJoinTuples for the current batch are
- * packed in 32kB buffers instead of pallocing each tuple individually.
+ * To reduce palloc/dsa_allocate overhead, the HashJoinTuples for the current
+ * batch are packed in 32kB buffers instead of pallocing each tuple
+ * individually.
*/
typedef struct HashMemoryChunkData
{
@@ -112,17 +129,120 @@ 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
+ {
+ dsa_pointer shared;
+ struct HashMemoryChunkData *private;
+ } 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_THRESHOLD (HASH_CHUNK_SIZE / 4)
+/*
+ * Read head position in a shared batch file.
+ */
+typedef struct HashJoinBatchPosition
+{
+ int fileno;
+ off_t offset;
+} HashJoinBatchPosition;
+
+/*
+ * The state exposed in shared memory for each participant to coordinate
+ * reading of batch files that it wrote.
+ */
+typedef struct HashJoinSharedBatchReader
+{
+ int batchno; /* the batch number we are currently reading */
+
+ LWLock lock; /* protects access to the members below */
+ bool error; /* has an IO error occurred? */
+ HashJoinBatchPosition head; /* shared read head for current batch */
+} HashJoinSharedBatchReader;
+
+/*
+ * The state exposed in shared memory by each participant allowing its batch
+ * files to be read by other participants.
+ */
+typedef struct HashJoinParticipantState
+{
+ /*
+ * Arrays of pointers to arrays of pointers to BufFileDesciptor objects
+ * exported by this participant. The descriptor for batch i is in slot
+ * i % (1 << fls(i - 1)) of the array at index fls(i).
+ *
+ * This arrangement means that we can modify future batches without
+ * moving/reallocating the current batch. The current batch is therefore
+ * immutable and accessible by other backends which need to read it.
+ */
+ dsa_pointer inner_batch_descriptors[32]; /* number of bits in batchno */
+ dsa_pointer outer_batch_descriptors[32];
+
+ /*
+ * The shared state used to coordinate reading from the current batch. We
+ * need separate objects for the outer and inner side, because in the
+ * probing phase some participants can be reading from the outer batch,
+ * while others can be reading from the inner side to preload the next
+ * batch.
+ */
+ HashJoinSharedBatchReader inner_batch_reader;
+ HashJoinSharedBatchReader outer_batch_reader;
+} HashJoinParticipantState;
+
+/*
+ * The state used by each backend to manage reading from batch files written
+ * by all participants.
+ */
+typedef struct HashJoinBatchReader
+{
+ int participant_number; /* read which participant's batch? */
+ int batchno; /* which batch are we reading? */
+ bool inner; /* inner or outer? */
+ HashJoinSharedBatchReader *shared; /* holder of the shared read head */
+ BufFile *file; /* the file opened in this backend */
+ HashJoinBatchPosition head; /* local read head position */
+} HashJoinBatchReader;
+
+/*
+ * State for a shared hash join table. Each backend participating in a hash
+ * join with a shared hash table also has a HashJoinTableData object in
+ * backend-private memory, which points to this shared state in the DSM
+ * segment.
+ */
+typedef struct SharedHashJoinTableData
+{
+ Barrier barrier; /* for synchronizing workers */
+ dsa_pointer primary_buckets; /* primary hash table */
+ dsa_pointer secondary_buckets; /* hash table for preloading next batch */
+ bool at_least_one_worker; /* did at least one worker join in time? */
+ int nbuckets;
+ int nbuckets_optimal;
+ pg_atomic_uint32 next_unmatched_bucket;
+ pg_atomic_uint64 total_primary_tuples;
+ pg_atomic_uint64 total_secondary_tuples;
+ dsa_pointer_atomic head_primary_chunk;
+ dsa_pointer_atomic head_secondary_chunk;
+ dsa_pointer_atomic chunks_to_rebucket;
+ int planned_participants; /* number of planned workers + leader */
+
+ /* state exposed by each participant for sharing batches */
+ HashJoinParticipantState participants[FLEXIBLE_ARRAY_MEMBER];
+} SharedHashJoinTableData;
+
+typedef union HashJoinBucketHead
+{
+ dsa_pointer_atomic shared;
+ HashJoinTuple private;
+} HashJoinBucketHead;
+
typedef struct HashJoinTableData
{
int nbuckets; /* # buckets in the in-memory hash table */
@@ -134,9 +254,11 @@ typedef struct HashJoinTableData
int log2_nbuckets_optimal; /* log2(nbuckets_optimal) */
/* buckets[i] is head of list of tuples in i'th in-memory bucket */
- struct HashJoinTupleData **buckets;
+ HashJoinBucketHead *buckets;
/* buckets array is per-batch storage, as are all the tuples */
+ HashJoinBucketHead *next_buckets; /* for preloading next batch */
+
bool keepNulls; /* true to store unmatchable NULL tuples */
bool skewEnabled; /* are we using skew optimization? */
@@ -185,7 +307,73 @@ typedef struct HashJoinTableData
MemoryContext batchCxt; /* context for this-batch-only storage */
/* used for dense allocation of tuples (into linked chunks) */
- HashMemoryChunk chunks; /* one list for the whole batch */
+ HashMemoryChunk primary_chunk; /* current chunk for this batch */
+ HashMemoryChunk secondary_chunk; /* current chunk for next batch */
+ HashMemoryChunk chunks_to_rebucket; /* after resizing table */
+ dsa_pointer primary_chunk_shared; /* DSA pointer to primary_chunk */
+ dsa_pointer secondary_chunk_shared; /* DSA pointer to secondary_chunk */
+
+ /* State for coordinating shared tables for parallel hash joins. */
+ dsa_area *area;
+ SharedHashJoinTableData *shared; /* the shared state */
+ int attached_at_phase; /* the phase this participant joined */
+ bool detached_early; /* did we decide to detach early? */
+ HashJoinBatchReader batch_reader; /* state for reading batches in */
} HashJoinTableData;
+/* Check if a HashJoinTable is shared by parallel workers. */
+#define HashJoinTableIsShared(table) ((table)->shared != NULL)
+
+/* The phases of parallel hash computation. */
+#define PHJ_PHASE_INIT 0
+#define PHJ_PHASE_CREATING 1
+#define PHJ_PHASE_HASHING 2
+#define PHJ_PHASE_RESIZING 3
+#define PHJ_PHASE_REBUCKETING 4
+#define PHJ_PHASE_PROBING 5 /* PHJ_PHASE_PROBING_BATCH(0) */
+#define PHJ_PHASE_UNMATCHED 6 /* PHJ_PHASE_UNMATCHED_BATCH(0) */
+
+/* The subphases for batches. */
+#define PHJ_SUBPHASE_PROMOTING 0
+#define PHJ_SUBPHASE_LOADING 1
+#define PHJ_SUBPHASE_PREPARING 2
+#define PHJ_SUBPHASE_PROBING 3
+#define PHJ_SUBPHASE_UNMATCHED 4
+
+/* The phases of parallel processing for batch(n). */
+#define PHJ_PHASE_PROMOTING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 4)
+#define PHJ_PHASE_LOADING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 3)
+#define PHJ_PHASE_PREPARING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 2)
+#define PHJ_PHASE_PROBING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 1)
+#define PHJ_PHASE_UNMATCHED_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 0)
+
+/* Phase number -> sub-phase within a batch. */
+#define PHJ_PHASE_TO_SUBPHASE(p) \
+ (((int)(p) - PHJ_PHASE_UNMATCHED + PHJ_SUBPHASE_UNMATCHED) % 5)
+
+/* Phase number -> batch number. */
+#define PHJ_PHASE_TO_BATCHNO(p) \
+ (((int)(p) - PHJ_PHASE_UNMATCHED + PHJ_SUBPHASE_UNMATCHED) / 5)
+
+/*
+ * Is a given phase one in which a new hash table array is being assigned by
+ * one elected backend? That includes initial creation, reallocation during
+ * resize, and promotion of secondary hash table to primary. Workers that
+ * show up and attach at an arbitrary time must wait such phases out before
+ * doing anything with the hash table.
+ */
+#define PHJ_PHASE_MUTATING_TABLE(p) \
+ ((p) == PHJ_PHASE_CREATING || \
+ (p) == PHJ_PHASE_RESIZING || \
+ (PHJ_PHASE_TO_BATCHNO(p) > 0 && \
+ PHJ_PHASE_TO_SUBPHASE(p) == PHJ_SUBPHASE_PROMOTING))
+
+/*
+ * Return the 'participant number' for a process participating in a parallel
+ * hash join. We give a number < hashtable->shared->planned_participants
+ * to each potential participant, including the leader.
+ */
+#define HashJoinParticipantNumber() \
+ (IsParallelWorker() ? ParallelWorkerNumber + 1 : 0)
+
#endif /* HASHJOIN_H */
diff --git a/src/include/executor/nodeHash.h b/src/include/executor/nodeHash.h
index 8cf6d15..b1e80f3 100644
--- a/src/include/executor/nodeHash.h
+++ b/src/include/executor/nodeHash.h
@@ -22,12 +22,12 @@ extern Node *MultiExecHash(HashState *node);
extern void ExecEndHash(HashState *node);
extern void ExecReScanHash(HashState *node);
-extern HashJoinTable ExecHashTableCreate(Hash *node, List *hashOperators,
+extern HashJoinTable ExecHashTableCreate(HashState *node, List *hashOperators,
bool keepNulls);
extern void ExecHashTableDestroy(HashJoinTable hashtable);
extern void ExecHashTableInsert(HashJoinTable hashtable,
TupleTableSlot *slot,
- uint32 hashvalue);
+ uint32 hashvalue, bool secondary);
extern bool ExecHashGetHashValue(HashJoinTable hashtable,
ExprContext *econtext,
List *hashkeys,
@@ -49,5 +49,7 @@ extern void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
int *numbatches,
int *num_skew_mcvs);
extern int ExecHashGetSkewBucket(HashJoinTable hashtable, uint32 hashvalue);
+extern void ExecHashUpdate(HashJoinTable hashtable);
+extern bool ExecHashCheckForEarlyExit(HashJoinTable hashtable);
#endif /* NODEHASH_H */
diff --git a/src/include/executor/nodeHashjoin.h b/src/include/executor/nodeHashjoin.h
index f24127a..d123e7e 100644
--- a/src/include/executor/nodeHashjoin.h
+++ b/src/include/executor/nodeHashjoin.h
@@ -14,15 +14,27 @@
#ifndef NODEHASHJOIN_H
#define NODEHASHJOIN_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
#include "storage/buffile.h"
+#include "storage/shm_toc.h"
extern HashJoinState *ExecInitHashJoin(HashJoin *node, EState *estate, int eflags);
extern TupleTableSlot *ExecHashJoin(HashJoinState *node);
extern void ExecEndHashJoin(HashJoinState *node);
extern void ExecReScanHashJoin(HashJoinState *node);
-extern void ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
- BufFile **fileptr);
+extern void ExecHashJoinSaveTuple(HashJoinTable hashtable,
+ MinimalTuple tuple, uint32 hashvalue,
+ int batchno, bool inner);
+extern void ExecHashJoinRewindBatches(HashJoinTable hashtable, int batchno);
+extern void ExecHashJoinOpenBatch(HashJoinTable hashtable,
+ int batchno, bool inner);
+extern void ExecHashJoinCloseBatch(HashJoinTable hashtable,
+ int batchno, bool inner);
+
+extern void ExecHashJoinEstimate(HashJoinState *state, ParallelContext *pcxt);
+extern void ExecHashJoinInitializeDSM(HashJoinState *state, ParallelContext *pcxt);
+extern void ExecHashJoinInitializeWorker(HashJoinState *state, shm_toc *toc);
#endif /* NODEHASHJOIN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 2fadf76..9ae55be 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1738,6 +1738,7 @@ typedef struct MergeJoinState
/* these structs are defined in executor/hashjoin.h: */
typedef struct HashJoinTupleData *HashJoinTuple;
typedef struct HashJoinTableData *HashJoinTable;
+typedef struct SharedHashJoinTableData *SharedHashJoinTable;
typedef struct HashJoinState
{
@@ -1759,6 +1760,7 @@ typedef struct HashJoinState
int hj_JoinState;
bool hj_MatchedOuter;
bool hj_OuterNotEmpty;
+ SharedHashJoinTable hj_sharedHashJoinTable;
} HashJoinState;
@@ -1982,6 +1984,9 @@ typedef struct HashState
HashJoinTable hashtable; /* hash table for the hashjoin */
List *hashkeys; /* list of ExprState nodes */
/* hashkeys is same as parent's hj_InnerHashKeys */
+
+ /* The following are the same as the parent's. */
+ SharedHashJoinTable shared_table_data;
} HashState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index e2fbc7d..e8f90d9 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -782,6 +782,7 @@ typedef struct Hash
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 */
+ bool shared_table; /* table shared by multiple workers? */
/* all other info is in the parent HashJoin node */
} Hash;
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..8b06551 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1258,6 +1258,16 @@ typedef struct MergePath
bool materialize_inner; /* add Materialize to inner? */
} MergePath;
+typedef enum
+{
+ /* Every worker builds its own private copy of the hash table. */
+ HASHPATH_TABLE_PRIVATE,
+ /* One worker builds a shared hash table, and all workers probe it. */
+ HASHPATH_TABLE_SHARED_SERIAL,
+ /* All workers build a shared hash table, and then probe it. */
+ HASHPATH_TABLE_SHARED_PARALLEL
+} HashPathTableType;
+
/*
* A hashjoin path has these fields.
*
@@ -1272,6 +1282,7 @@ typedef struct HashPath
JoinPath jpath;
List *path_hashclauses; /* join clauses used for hashing */
int num_batches; /* number of batches expected */
+ HashPathTableType table_type; /* level of sharedness */
} HashPath;
/*
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..7bb0d1d 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -24,6 +24,7 @@
#define DEFAULT_SEQ_PAGE_COST 1.0
#define DEFAULT_RANDOM_PAGE_COST 4.0
#define DEFAULT_CPU_TUPLE_COST 0.01
+#define DEFAULT_CPU_SHARED_TUPLE_COST 0.0
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
#define DEFAULT_PARALLEL_TUPLE_COST 0.1
@@ -48,6 +49,7 @@ typedef enum
extern PGDLLIMPORT double seq_page_cost;
extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
+extern PGDLLIMPORT double cpu_shared_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
extern PGDLLIMPORT double parallel_tuple_cost;
@@ -144,7 +146,8 @@ extern void initial_cost_hashjoin(PlannerInfo *root,
List *hashclauses,
Path *outer_path, Path *inner_path,
SpecialJoinInfo *sjinfo,
- SemiAntiJoinFactors *semifactors);
+ SemiAntiJoinFactors *semifactors,
+ HashPathTableType table_type);
extern void final_cost_hashjoin(PlannerInfo *root, HashPath *path,
JoinCostWorkspace *workspace,
SpecialJoinInfo *sjinfo,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..5f4ca87 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -134,7 +134,8 @@ extern HashPath *create_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *restrict_clauses,
Relids required_outer,
- List *hashclauses);
+ List *hashclauses,
+ HashPathTableType table_type);
extern ProjectionPath *create_projection_path(PlannerInfo *root,
RelOptInfo *rel,
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 0b85b7a..0157d52 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -785,7 +785,22 @@ typedef enum
WAIT_EVENT_MQ_SEND,
WAIT_EVENT_PARALLEL_FINISH,
WAIT_EVENT_SAFE_SNAPSHOT,
- WAIT_EVENT_SYNC_REP
+ WAIT_EVENT_SYNC_REP,
+ WAIT_EVENT_HASH_CREATING,
+ WAIT_EVENT_HASH_HASHING,
+ WAIT_EVENT_HASH_RESIZING,
+ WAIT_EVENT_HASH_REBUCKETING,
+ WAIT_EVENT_HASH_INIT,
+ WAIT_EVENT_HASH_DESTROY,
+ WAIT_EVENT_HASH_UNMATCHED,
+ WAIT_EVENT_HASH_PROMOTING,
+ WAIT_EVENT_HASHJOIN_PROMOTING,
+ WAIT_EVENT_HASHJOIN_PROBING,
+ WAIT_EVENT_HASHJOIN_LOADING,
+ WAIT_EVENT_HASHJOIN_PREPARING,
+ WAIT_EVENT_HASHJOIN_REWINDING,
+ WAIT_EVENT_HASHJOIN_REWINDING2, /* TODO: rename me */
+ WAIT_EVENT_HASHJOIN_REWINDING3 /* TODO: rename me */
} WaitEventIPC;
/* ----------
diff --git a/src/include/storage/buffile.h b/src/include/storage/buffile.h
index 809e596..044262d 100644
--- a/src/include/storage/buffile.h
+++ b/src/include/storage/buffile.h
@@ -30,12 +30,17 @@
typedef struct BufFile BufFile;
+typedef struct BufFileDescriptor BufFileDescriptor;
+
/*
* prototypes for functions in buffile.c
*/
extern BufFile *BufFileCreateTemp(bool interXact);
extern void BufFileClose(BufFile *file);
+extern BufFileDescriptor *BufFileExport(BufFile *file);
+extern BufFile *BufFileImport(BufFileDescriptor *descriptor);
+extern size_t BufFileDescriptorSize(const BufFileDescriptor *descriptor);
extern size_t BufFileRead(BufFile *file, void *ptr, size_t size);
extern size_t BufFileWrite(BufFile *file, void *ptr, size_t size);
extern int BufFileSeek(BufFile *file, int fileno, off_t offset, int whence);
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 951e421..7af6e04 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -236,6 +236,8 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_LOCK_MANAGER,
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_EXEC_AREA,
+ LWTRANCHE_PARALLEL_HASH_JOIN_INNER_BATCH_READER,
+ LWTRANCHE_PARALLEL_HASH_JOIN_OUTER_BATCH_READER,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
On Thu, Nov 3, 2016 at 4:19 PM, Thomas Munro <thomas.munro@enterprisedb.com>
wrote:
Obviously I'm actively working on developing and stabilising all this.
Some of the things I'm working on are: work_mem accounting, batch
increases, rescans and figuring out if the resource management for
those BufFiles is going to work. There are quite a lot of edge cases
some of which I'm still figuring out, but I feel like this approach is
workable. At this stage I want to share what I'm doing to see if
others have feedback, ideas, blood curdling screams of horror, etc. I
will have better patches and a set of test queries soon. Thanks for
reading.
This patch doesn't receive any review. Patch is not applying properly to
HEAD.
Moved to next CF with "waiting on author" status.
Regards,
Hari Babu
Fujitsu Australia
On Sat, Dec 3, 2016 at 1:38 AM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote:
Moved to next CF with "waiting on author" status.
Unfortunately it's been a bit trickier than I anticipated to get the
interprocess batch file sharing and hash table shrinking working
correctly and I don't yet have a new patch in good enough shape to
post in time for the January CF. More soon.
--
Thomas Munro
http://www.enterprisedb.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sat, Dec 31, 2016 at 2:52 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
Unfortunately it's been a bit trickier than I anticipated to get the
interprocess batch file sharing and hash table shrinking working
correctly and I don't yet have a new patch in good enough shape to
post in time for the January CF. More soon.
I noticed a bug in your latest revision:
+ /* + * In HJ_NEED_NEW_OUTER, we already selected the current inner batch for + * reading from. If there is a shared hash table, we may have already + * partially loaded the hash table in ExecHashJoinPreloadNextBatch. + */ + Assert(hashtable->batch_reader.batchno = curbatch); + Assert(hashtable->batch_reader.inner);
Obviously this isn't supposed to be an assignment.
--
Peter Geoghegan
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Jan 2, 2017 at 3:17 PM, Peter Geoghegan <pg@heroku.com> wrote:
I noticed a bug in your latest revision:
+ /* + * In HJ_NEED_NEW_OUTER, we already selected the current inner batch for + * reading from. If there is a shared hash table, we may have already + * partially loaded the hash table in ExecHashJoinPreloadNextBatch. + */ + Assert(hashtable->batch_reader.batchno = curbatch); + Assert(hashtable->batch_reader.inner);Obviously this isn't supposed to be an assignment.
Right, thanks! I will post a new rebased version soon with that and
some other nearby problems fixed.
--
Thomas Munro
http://www.enterprisedb.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Jan 3, 2017 at 10:53 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
I will post a new rebased version soon with that and
some other nearby problems fixed.
Here is a new WIP patch. I have plenty of things to tidy up (see note
at end), but the main ideas are now pretty clear and I'd appreciate
some feedback. The main changes since the last patch, other than
debugging, are:
* the number of batches now increases if work_mem would be exceeded;
the work of 'shrinking' the hash table in memory in that case is done
in parallel
* work_mem accounting is done at chunk level, instead of tuples
* interlocking has been rethought
Previously, I had some ideas about using some lock free tricks for
managing chunks of memory, but you may be relieved to hear that I
abandoned those plans. Now, atomic ops are used only for one thing:
pushing tuples into the shared hash table buckets. An LWLock called
'chunk_lock' protects various linked lists of chunks of memory, and
also the shared work_mem accounting. The idea is that backends can
work independently on HASH_CHUNK_SIZE blocks of tuples at a time in
between needing to acquire that lock briefly. Also, there is now a
second barrier, used to coordinate hash table shrinking. This can
happen any number of times during PHJ_PHASE_HASHING and
PHJ_PHASE_LOADING_BATCH(n) phases as required to stay under work_mem,
so it needed to be a separate barrier.
The communication in this patch is a bit more complicated than other
nearby parallel query projects I've looked at; probably the worst bit
is the leader deadlock avoidance stuff (see
ExecHashCheckForEarlyExit), and the second worst bit is probably the
switch statements for allowing participants to show up late and get in
sync, which makes that other problem even more annoying; without those
problems and with just the right kind of reusable shared tuplestore,
this would be a vastly simpler patch. Those are not really
fundamental problems of parallel joins using a shared hash tables, but
they're problems I don't have a better solution to right now.
Stepping back a bit, I am aware of the following approaches to hash
join parallelism:
1. Run the inner plan and build a private hash table in each
participant, and then scatter the outer plan arbitrarily across
participants. This is what 9.6 does, and it's a good plan for small
hash tables with fast inner plans, but a terrible plan for expensive
or large inner plans. Communication overhead: zero; CPU overhead:
runs the inner plan in k workers simultaneously; memory overhead:
builds k copies of the hashtable; disk overhead: may need to spill k
copies of all batches to disk if work_mem exceeded; restrictions:
Can't do right/full joins because no shared 'matched' flags.
2. Run a partition-wise hash join[1]/messages/by-id/CAFjFpRfQ8GrQvzp3jA2wnLqrHmaXna-urjm_UY9BqXj=EaDTSA@mail.gmail.com. Communication overhead: zero;
CPU overhead: zero; memory overhead: zero; disk overhead: zero;
restrictions: the schema must include compatible partition keys, and
potential parallelism is limited by the number of partitions.
3. Repartition the data on the fly, and then run a partition-wise
hash join. Communication overhead: every tuple on at least one and
possibly both sides must be rerouted to the correct participant; CPU
overhead: zero, once repartitioning is done; memory overhead: none;
disk overhead: may need to spill partitions to disk if work_mem is
exceeded
4. Scatter both the inner and outer plans arbitrarily across
participants (ie uncorrelated partitioning), and build a shared hash
table. Communication overhead: synchronisation of build/probe phases,
but no tuple rerouting; CPU overhead: none; memory overhead: none;
disk overhead: may need to spill batches to disk; restrictions: none
in general, but currently we have to drop the leader after the first
batch of a multi-batch join due to our consumer/producer leader
problem mentioned in earlier messages.
We have 1. This proposal aims to provide 4. It seems we have 2 on
the way (that technique works for all 3 join algorithms without any
changes to the join operators and looks best by any measure, but is
limited by the user's schema, ie takes careful planning on the user's
part instead of potentially helping any join). Other databases
including SQL Server offer 3. I suspect that 4 is probably a better
fit than 3 for Postgres today, because the communication overhead of
shovelling nearly all tuples through extra tuple queues to route them
to the right hash table would surely be very high, though I can see
that it's very attractive to have a reusable tuple repartitioning
operator and then run k disjoint communication-free joins (again,
without code change to the join operator, and to the benefit of all
join operators).
About the shared batch reading code: this patch modifies BufFile so
that a temporary file can be shared read-only with other participants,
and then introduces a mechanism for coordinating shared reads. Each
worker starts out reading all the tuples from the file that it wrote,
before attempting to steal tuples from the files written by other
participants, until there are none left anywhere. In the best case
they all write out and then read back in just their own files with
minimal contention, and contention rises as tuples are less evenly
distributed among participants, but we never quite get the best case
because the leader always leaves behind a bunch of batches for the
others to deal with when it quits early. Maybe I should separate all
the batch reader stuff into another patch so it doesn't clutter the
hash join code up so much? I will start reviewing Parallel Tuplesort
shortly, which includes some related ideas.
Some assorted notes on the status: I need to do some thinking about
the file cleanup logic: both explicit deletes at the earliest possible
time, and failure/error paths. Currently the creator of each file is
responsible for cleaning it up, but I guess if the creator aborts
early the file disappears underneath the others' feet, and then I
guess they might raise a confusing error report that races against the
root cause error report; I'm looking into that. Rescans and skew
buckets not finished yet. The new chunk-queue based
ExecScanHashTableForUnmatched isn't tested yet (it replaces and
earlier version that was doing a bucket-by-bucket parallel scan).
There are several places where I haven't changed the private hash
table code to match the shared version because I'm not sure about
that, in particular the idea of chunk-based accounting (which happens
to be convenient for this code, but I also believe it to be more
correct). I'm still trying to decide how to report the hash table
tuple count and size: possibly the grand totals. Generally I need to
do some tidying and provide a suite of queries that hits interesting
cases. I hope to move on these things fairly quickly now that I've
got the hash table resizing and batch sharing stuff working (a puzzle
that kept me very busy for a while) though I'm taking a break for a
bit to do some reviewing.
The test query I've been looking at recently is TPCH Q9. With scale
1GB and work_mem = 64KB, I get a query plan that includes three
different variants of Hash node: Hash (run in every backend, duplicate
hash tables), Shared Hash (run in just one backend, but allowed to use
the sum of work_mem of all the backends, so usually wins by avoiding
batching), and Parallel Shared Hash (run in parallel and using sum of
work_mem). As an anecdatum, I see around 2.5x speedup against master,
using only 2 workers in both cases, though it seems to be bimodal,
either 2x or 2.8x, which I expect has something to do with that leader
exit stuff and I'm looking into that.. More on performance soon.
Thanks for reading!
[1]: /messages/by-id/CAFjFpRfQ8GrQvzp3jA2wnLqrHmaXna-urjm_UY9BqXj=EaDTSA@mail.gmail.com
--
Thomas Munro
http://www.enterprisedb.com
Attachments:
parallel-hash-v3.patchapplication/octet-stream; name=parallel-hash-v3.patchDownload
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index c762fb0..43e85fc 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -1023,7 +1023,10 @@ ExplainNode(PlanState *planstate, List *ancestors,
pname = sname = "Limit";
break;
case T_Hash:
- pname = sname = "Hash";
+ if (((Hash *) plan)->shared_table)
+ pname = sname = "Shared Hash";
+ else
+ pname = sname = "Hash";
break;
default:
pname = sname = "???";
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 86d9fb5..361d56a 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -27,6 +27,7 @@
#include "executor/executor.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
+#include "executor/nodeHashjoin.h"
#include "executor/nodeSeqscan.h"
#include "executor/tqueue.h"
#include "nodes/nodeFuncs.h"
@@ -203,6 +204,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
break;
+ case T_HashJoinState:
+ ExecHashJoinEstimate((HashJoinState *) planstate,
+ e->pcxt);
+ break;
default:
break;
}
@@ -255,6 +260,9 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
break;
+ case T_HashJoinState:
+ ExecHashJoinInitializeDSM((HashJoinState *) planstate,
+ d->pcxt);
default:
break;
}
@@ -731,6 +739,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
break;
+ case T_HashJoinState:
+ ExecHashJoinInitializeWorker((HashJoinState *) planstate,
+ toc);
+ break;
default:
break;
}
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index b8edd36..5c402bb 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -806,6 +806,9 @@ ExecShutdownNode(PlanState *node)
case T_GatherState:
ExecShutdownGather((GatherState *) node);
break;
+ case T_HashJoinState:
+ ExecShutdownHashJoin((HashJoinState *) node);
+ break;
default:
break;
}
diff --git a/src/backend/executor/nodeHash.c b/src/backend/executor/nodeHash.c
index 11db08f..5301bc0 100644
--- a/src/backend/executor/nodeHash.c
+++ b/src/backend/executor/nodeHash.c
@@ -25,6 +25,7 @@
#include <limits.h>
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "catalog/pg_statistic.h"
#include "commands/tablespace.h"
#include "executor/execdebug.h"
@@ -32,14 +33,17 @@
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
#include "miscadmin.h"
+#include "pgstat.h"
+#include "port/atomics.h"
#include "utils/dynahash.h"
#include "utils/memutils.h"
#include "utils/lsyscache.h"
+#include "utils/probes.h"
#include "utils/syscache.h"
-
static void ExecHashIncreaseNumBatches(HashJoinTable hashtable);
static void ExecHashIncreaseNumBuckets(HashJoinTable hashtable);
+static void ExecHashShrink(HashJoinTable hashtable);
static void ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node,
int mcvsToUse);
static void ExecHashSkewTableInsert(HashJoinTable hashtable,
@@ -47,8 +51,28 @@ static void ExecHashSkewTableInsert(HashJoinTable hashtable,
uint32 hashvalue,
int bucketNumber);
static void ExecHashRemoveNextSkewBucket(HashJoinTable hashtable);
+static void ExecHashTableComputeOptimalBuckets(HashJoinTable hashtable);
+
+static HashJoinTuple next_tuple_in_bucket(HashJoinTable table,
+ HashJoinTuple tuple);
+static HashJoinTuple first_tuple_in_skew_bucket(HashJoinTable table,
+ int skew_bucket_no);
+static HashJoinTuple first_tuple_in_skew_bucket(HashJoinTable table,
+ int bucket_no);
+static void insert_tuple_into_bucket(HashJoinTable table, int bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_pointer);
+static void insert_tuple_into_skew_bucket(HashJoinTable table,
+ int bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_pointer);
static void *dense_alloc(HashJoinTable hashtable, Size size);
+static void *dense_alloc_shared(HashJoinTable hashtable, Size size,
+ dsa_pointer *chunk_shared,
+ bool secondary,
+ bool force);
+
/* ----------------------------------------------------------------
* ExecHash
@@ -64,6 +88,98 @@ ExecHash(HashState *node)
}
/* ----------------------------------------------------------------
+ * ExecHashCheckForEarlyExit
+ *
+ * return true if this process needs to abandon work on the
+ * hash join to avoid a deadlock
+ * ----------------------------------------------------------------
+ */
+bool
+ExecHashCheckForEarlyExit(HashJoinTable hashtable)
+{
+ /*
+ * The golden rule of leader deadlock avoidance: since leader processes
+ * have two separate roles, namely reading from worker queues AND executing
+ * the same plan as workers, we must never allow a leader to wait for
+ * workers if there is any possibility those workers have emitted tuples.
+ * Otherwise we could get into a situation where a worker fills up its
+ * output tuple queue and begins waiting for the leader to read, while
+ * the leader is busy waiting for the worker.
+ *
+ * Parallel hash joins with shared tables are inherently susceptible to
+ * such deadlocks because there are points at which all participants must
+ * wait (you can't start check for unmatched tuples in the hash table until
+ * probing has completed in all workers, etc).
+ *
+ * So we follow these rules:
+ *
+ * 1. If there are workers participating, the leader MUST NOT not
+ * participate in any further work after probing the first batch, so
+ * that it never has to wait for workers that might have emitted
+ * tuples.
+ *
+ * 2. If there are no workers participating, the leader MUST run all the
+ * batches to completion, because that's the only way for the join
+ * to complete. There is no deadlock risk if there are no workers.
+ *
+ * 3. Workers MUST NOT participate if the hashing phase has finished by
+ * the time they have joined, so that the leader can reliably determine
+ * whether there are any workers running when it comes to the point
+ * where it must choose between 1 and 2.
+ *
+ * In other words, if the leader makes it all the way through hashing and
+ * probing before any workers show up, then the leader will run the whole
+ * hash join on its own. If workers do show up any time before hashing is
+ * finished, the leader will stop executing the join after helping probe
+ * the first batch. In the unlikely event of the first worker showing up
+ * after the leader has finished hashing, it will exit because it's too
+ * late, the leader has already decided to do all the work alone.
+ */
+
+ if (!IsParallelWorker())
+ {
+ /* Running in the leader process. */
+ if (BarrierPhase(&hashtable->shared->barrier) >= PHJ_PHASE_PROBING &&
+ hashtable->shared->at_least_one_worker)
+ {
+ /* Abandon ship due to rule 1. There are workers running. */
+ TRACE_POSTGRESQL_HASH_LEADER_EARLY_EXIT();
+ return true;
+ }
+ else
+ {
+ /*
+ * Continue processing due to rule 2. There are no workers, and
+ * any workers that show up later will abandon ship.
+ */
+ }
+ }
+ else
+ {
+ /* Running in a worker process. */
+ if (hashtable->attached_at_phase < PHJ_PHASE_PROBING)
+ {
+ /*
+ * Advertise that there are workers, so that the leader can
+ * choose between rules 1 and 2. It's OK that several workers can
+ * write to this variable without immediately memory
+ * synchronization, because the leader will only read it in a later
+ * phase (see above).
+ */
+ hashtable->shared->at_least_one_worker = true;
+ }
+ else
+ {
+ /* Abandon ship due to rule 3. */
+ TRACE_POSTGRESQL_HASH_WORKER_EARLY_EXIT();
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/* ----------------------------------------------------------------
* MultiExecHash
*
* build hash table for hashjoin, doing partitioning if more
@@ -79,6 +195,7 @@ MultiExecHash(HashState *node)
TupleTableSlot *slot;
ExprContext *econtext;
uint32 hashvalue;
+ Barrier *barrier = NULL;
/* must provide our own instrumentation support */
if (node->ps.instrument)
@@ -90,6 +207,63 @@ MultiExecHash(HashState *node)
outerNode = outerPlanState(node);
hashtable = node->hashtable;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Synchronize parallel hash table builds. At this stage we know that
+ * the shared hash table has been created, but we don't know if our
+ * peers are still in MultiExecHash and if so how far through. We use
+ * the phase to synchronize with them.
+ */
+ barrier = &hashtable->shared->barrier;
+
+ switch (BarrierPhase(barrier))
+ {
+ case PHJ_PHASE_BEGINNING:
+ /* ExecHashTableCreate already handled this phase. */
+ Assert(false);
+ case PHJ_PHASE_CREATING:
+ /* Wait for serial phase, and then either hash or wait. */
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_CREATING))
+ goto hash;
+ else if (node->ps.plan->parallel_aware)
+ goto hash;
+ else
+ goto post_hash;
+ case PHJ_PHASE_HASHING:
+ /* Hashing is already underway. Can we join in? */
+ if (node->ps.plan->parallel_aware)
+ goto hash;
+ else
+ goto post_hash;
+ case PHJ_PHASE_RESIZING:
+ /* Can't help with serial phase. */
+ goto post_resize;
+ case PHJ_PHASE_REBUCKETING:
+ /* Rebucketing is in progress. Let's help do that. */
+ goto rebucket;
+ default:
+ /* The hash table building work is already finished. */
+ goto finish;
+ }
+ }
+
+ hash:
+ TRACE_POSTGRESQL_HASH_HASHING_START();
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Make sure our local hashtable is up-to-date so we can hash. */
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_HASHING);
+ ExecHashUpdate(hashtable);
+
+ /*
+ * Attach to the second barrier that is just used for coordinating
+ * shrinking during the hashing phase, in case we run out of work_mem.
+ */
+ BarrierAttach(&hashtable->shared->shrink_barrier);
+ }
+
/*
* set expression context
*/
@@ -123,22 +297,106 @@ MultiExecHash(HashState *node)
else
{
/* Not subject to skew optimization, so insert normally */
- ExecHashTableInsert(hashtable, slot, hashvalue);
+ ExecHashTableInsert(hashtable, slot, hashvalue, false);
}
- hashtable->totalTuples += 1;
+ /*
+ * Shared tuple counters are managed by dense_alloc_shared. For
+ * private hash tables we maintain the counter here.
+ */
+ if (!HashJoinTableIsShared(hashtable))
+ hashtable->totalTuples += 1;
}
}
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Detach from the shrink barrier. */
+ BarrierDetach(&hashtable->shared->shrink_barrier);
+ }
+
+ TRACE_POSTGRESQL_HASH_HASHING_DONE();
+
+ post_hash:
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ bool elected_to_resize;
+
+ /*
+ * Wait for all backends to finish hashing. If only one worker is
+ * running the hashing phase because of a non-partial inner plan, the
+ * other workers will pile up here waiting. If multiple worker are
+ * hashing, they should finish close to each other in time.
+ *
+ * TODO: Even if only one backend is allowed to run the plan, other
+ * backends might as well stand ready to help with rebatching work if
+ * the need arises. Maybe we need a way to 'arrive' at a barrier, but
+ * not block, then a way to loop on another condition variable,
+ * running ExecHashShrink each time we're woken, and break when all
+ * partipants have arrived at the barrier (ie when
+ * BarrierPhase(barrier) reports that the phase has advanced).
+ */
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_HASHING);
+ elected_to_resize = BarrierWait(barrier, WAIT_EVENT_HASH_HASHING);
+ /*
+ * Resizing is a serial phase. All but one should skip ahead to
+ * rebucketing, but all workers should update their copy of the shared
+ * tuple count with the final total first.
+ */
+ /*
+ hashtable->totalTuples =
+ pg_atomic_read_u64(&hashtable->shared->total_primary_tuples);
+ */
+ if (!elected_to_resize)
+ goto post_resize;
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_RESIZING);
+ }
+
/* resize the hash table if needed (NTUP_PER_BUCKET exceeded) */
- if (hashtable->nbuckets != hashtable->nbuckets_optimal)
- ExecHashIncreaseNumBuckets(hashtable);
+ ExecHashIncreaseNumBuckets(hashtable);
+
+ post_resize:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_RESIZING);
+ BarrierWait(&hashtable->shared->barrier,
+ WAIT_EVENT_HASH_RESIZING);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_REBUCKETING);
+ }
+
+ rebucket:
+ /* If the table was resized, insert tuples into the new buckets. */
+ ExecHashUpdate(hashtable);
+ ExecHashRebucket(hashtable);
/* Account for the buckets in spaceUsed (reported in EXPLAIN ANALYZE) */
- hashtable->spaceUsed += hashtable->nbuckets * sizeof(HashJoinTuple);
+ hashtable->spaceUsed += hashtable->nbuckets * sizeof(HashJoinBucketHead);
if (hashtable->spaceUsed > hashtable->spacePeak)
hashtable->spacePeak = hashtable->spaceUsed;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_REBUCKETING);
+ BarrierWait(barrier, WAIT_EVENT_HASH_REBUCKETING);
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_PROBING);
+ }
+
+ finish:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * All hashing work has finished. The other workers may be probing or
+ * processing unmatched tuples for the initial batch, or dealing with
+ * later batches. The next synchronization point is in ExecHashJoin's
+ * HJ_BUILD_HASHTABLE case, which will figure that out and synchronize
+ * its local state machine with the parallel processing group's phase.
+ */
+ Assert(BarrierPhase(barrier) >= PHJ_PHASE_PROBING);
+ ExecHashUpdate(hashtable);
+ }
+
/* must provide our own instrumentation support */
+ /* TODO: report only the tuples that WE hashed here? */
if (node->ps.instrument)
InstrStopNode(node->ps.instrument, hashtable->totalTuples);
@@ -243,10 +501,13 @@ ExecEndHash(HashState *node)
* ----------------------------------------------------------------
*/
HashJoinTable
-ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
+ExecHashTableCreate(HashState *state, List *hashOperators, bool keepNulls)
{
+ Hash *node;
HashJoinTable hashtable;
+ SharedHashJoinTable shared_hashtable;
Plan *outerNode;
+ size_t space_allowed;
int nbuckets;
int nbatch;
int num_skew_mcvs;
@@ -261,10 +522,15 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
* "outer" subtree of this node, but the inner relation of the hashjoin).
* Compute the appropriate size of the hash table.
*/
+ node = (Hash *) state->ps.plan;
outerNode = outerPlan(node);
-
+ shared_hashtable = state->shared_table_data;
ExecChooseHashTableSize(outerNode->plan_rows, outerNode->plan_width,
OidIsValid(node->skewTable),
+ shared_hashtable != NULL,
+ shared_hashtable != NULL ?
+ shared_hashtable->planned_participants - 1 : 0,
+ &space_allowed,
&nbuckets, &nbatch, &num_skew_mcvs);
/* nbuckets must be a power of 2 */
@@ -301,11 +567,19 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
hashtable->outerBatchFile = NULL;
hashtable->spaceUsed = 0;
hashtable->spacePeak = 0;
- hashtable->spaceAllowed = work_mem * 1024L;
+ hashtable->spaceAllowed = space_allowed;
hashtable->spaceUsedSkew = 0;
hashtable->spaceAllowedSkew =
hashtable->spaceAllowed * SKEW_WORK_MEM_PERCENT / 100;
- hashtable->chunks = NULL;
+ hashtable->chunk = NULL;
+ hashtable->chunk_preload = NULL;
+ hashtable->chunks_to_rebucket = NULL;
+ hashtable->chunk_shared = InvalidDsaPointer;
+ hashtable->chunk_preload_shared = InvalidDsaPointer;
+ hashtable->area = state->ps.state->es_query_dsa;
+ hashtable->shared = state->shared_table_data;
+ hashtable->preloaded_spare_tuple = false;
+ hashtable->detached_early = false;
#ifdef HJDEBUG
printf("Hashjoin %p: initial nbatch = %d, nbuckets = %d\n",
@@ -340,7 +614,7 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
/*
* Create temporary memory contexts in which to keep the hashtable working
- * storage. See notes in executor/hashjoin.h.
+ * storage if using private hash table. See notes in executor/hashjoin.h.
*/
hashtable->hashCxt = AllocSetContextCreate(CurrentMemoryContext,
"HashTableContext",
@@ -368,23 +642,95 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
PrepareTempTablespaces();
}
- /*
- * Prepare context for the first-scan space allocations; allocate the
- * hashbucket array therein, and set each bucket "empty".
- */
- MemoryContextSwitchTo(hashtable->batchCxt);
+ MemoryContextSwitchTo(oldcxt);
- hashtable->buckets = (HashJoinTuple *)
- palloc0(nbuckets * sizeof(HashJoinTuple));
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier;
- /*
- * Set up for skew optimization, if possible and there's a need for more
- * than one batch. (In a one-batch join, there's no point in it.)
- */
- if (nbatch > 1)
- ExecHashBuildSkewHash(hashtable, node, num_skew_mcvs);
+ /*
+ * Attach to the barrier. The corresponding detach operation is in
+ * ExecHashTableDestroy.
+ */
+ barrier = &hashtable->shared->barrier;
+ hashtable->attached_at_phase = BarrierAttach(barrier);
- MemoryContextSwitchTo(oldcxt);
+ /*
+ * So far we have no idea whether there are any other participants, and
+ * if so, what phase they are working on. The only thing we care about
+ * at this point is whether someone has already created the shared
+ * hash table yet. If not, one backend will be elected to do that
+ * now.
+ */
+ if (BarrierPhase(barrier) == PHJ_PHASE_BEGINNING)
+ {
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_BEGINNING))
+ {
+ /* Serial phase: create the hash tables */
+ Size bytes;
+ HashJoinBucketHead *buckets;
+ int i;
+ SharedHashJoinTable shared;
+ dsa_area *area;
+
+ shared = hashtable->shared;
+ area = hashtable->area;
+ bytes = nbuckets * sizeof(HashJoinBucketHead);
+
+ /* Allocate the hash table buckets. */
+ shared->buckets = dsa_allocate(area, bytes);
+ if (!DsaPointerIsValid(shared->buckets))
+ ereport(ERROR,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("out of memory")));
+
+ /* Initialize the hash table buckets to empty. */
+ buckets = dsa_get_address(area, shared->buckets);
+ for (i = 0; i < nbuckets; ++i)
+ dsa_pointer_atomic_init(&buckets[i].shared,
+ InvalidDsaPointer);
+
+ /* Initialize the rest of parallel_state. */
+ hashtable->shared->nbuckets = nbuckets;
+ hashtable->shared->nbatch = nbatch;
+ hashtable->shared->size = bytes;
+ hashtable->shared->size_preloaded = 0;
+ ExecHashJoinRewindBatches(hashtable, 0);
+
+ /* TODO: ExecHashBuildSkewHash */
+
+ /*
+ * The backend-local pointers in hashtable will be set up by
+ * ExecHashUpdate, at each point where they might have
+ * changed.
+ */
+ }
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_CREATING);
+ /* The next synchronization point is in MultiExecHash. */
+ }
+ }
+ else
+ {
+ /*
+ * Prepare context for the first-scan space allocations; allocate the
+ * hashbucket array therein, and set each bucket "empty".
+ */
+ MemoryContextSwitchTo(hashtable->batchCxt);
+
+ hashtable->buckets = (HashJoinBucketHead *)
+ palloc0(nbuckets * sizeof(HashJoinBucketHead));
+
+ MemoryContextSwitchTo(oldcxt);
+
+ /*
+ * Set up for skew optimization, if possible and there's a need for
+ * more than one batch. (In a one-batch join, there's no point in
+ * it.)
+ */
+ if (nbatch > 1)
+ ExecHashBuildSkewHash(hashtable, node, num_skew_mcvs);
+ }
return hashtable;
}
@@ -402,6 +748,8 @@ ExecHashTableCreate(Hash *node, List *hashOperators, bool keepNulls)
void
ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
+ bool shared, int parallel_workers,
+ size_t *space_allowed,
int *numbuckets,
int *numbatches,
int *num_skew_mcvs)
@@ -432,9 +780,15 @@ ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
inner_rel_bytes = ntuples * tupsize;
/*
- * Target in-memory hashtable size is work_mem kilobytes.
+ * Target in-memory hashtable size is work_mem kilobytes. Shared hash
+ * tables are allowed to multiply work_mem by the number of participants,
+ * since other non-shared memory based plans allow each participant to use
+ * work_mem for the same total.
*/
hash_table_bytes = work_mem * 1024L;
+ if (shared && parallel_workers > 0)
+ hash_table_bytes *= parallel_workers + 1; /* one for the leader */
+ *space_allowed = hash_table_bytes;
/*
* If skew optimization is possible, estimate the number of skew buckets
@@ -481,8 +835,8 @@ ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
* Note that both nbuckets and nbatch must be powers of 2 to make
* ExecHashGetBucketAndBatch fast.
*/
- max_pointers = (work_mem * 1024L) / sizeof(HashJoinTuple);
- max_pointers = Min(max_pointers, MaxAllocSize / sizeof(HashJoinTuple));
+ max_pointers = (work_mem * 1024L) / sizeof(HashJoinBucketHead);
+ max_pointers = Min(max_pointers, MaxAllocSize / sizeof(HashJoinBucketHead));
/* If max_pointers isn't a power of 2, must round it down to one */
mppow2 = 1L << my_log2(max_pointers);
if (max_pointers != mppow2)
@@ -504,7 +858,7 @@ ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
* If there's not enough space to store the projected number of tuples and
* the required bucket headers, we will need multiple batches.
*/
- bucket_bytes = sizeof(HashJoinTuple) * nbuckets;
+ bucket_bytes = sizeof(HashJoinBucketHead) * nbuckets;
if (inner_rel_bytes + bucket_bytes > hash_table_bytes)
{
/* We'll need multiple batches */
@@ -519,12 +873,12 @@ ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
* NTUP_PER_BUCKET tuples, whose projected size already includes
* overhead for the hash code, pointer to the next tuple, etc.
*/
- bucket_size = (tupsize * NTUP_PER_BUCKET + sizeof(HashJoinTuple));
+ bucket_size = (tupsize * NTUP_PER_BUCKET + sizeof(HashJoinBucketHead));
lbuckets = 1L << my_log2(hash_table_bytes / bucket_size);
lbuckets = Min(lbuckets, max_pointers);
nbuckets = (int) lbuckets;
nbuckets = 1 << my_log2(nbuckets);
- bucket_bytes = nbuckets * sizeof(HashJoinTuple);
+ bucket_bytes = nbuckets * sizeof(HashJoinBucketHead);
/*
* Buckets are simple pointers to hashjoin tuples, while tupsize
@@ -564,6 +918,31 @@ ExecHashTableDestroy(HashJoinTable hashtable)
{
int i;
+ /* Detached, if we haven't already. */
+ if (HashJoinTableIsShared(hashtable) && !hashtable->detached_early)
+ {
+ Barrier *barrier = &hashtable->shared->barrier;
+
+ /*
+ * TODO: Can we just detach if there is only one batch, but wait here
+ * if there is more than one (to make sure batch files created by this
+ * participant are not deleted)? When detaching, the last one to
+ * detach should do the cleanup work, and/or leave things in the right
+ * state for rescanning.
+ */
+
+ if (BarrierWait(barrier, WAIT_EVENT_HASH_DESTROY))
+ {
+ /* Serial: free the tables */
+ if (DsaPointerIsValid(hashtable->shared->buckets))
+ {
+ dsa_free(hashtable->area, hashtable->shared->buckets);
+ hashtable->shared->buckets = InvalidDsaPointer;
+ }
+ }
+ BarrierDetach(&hashtable->shared->barrier);
+ }
+
/*
* Make sure all the temp files are closed. We skip batch 0, since it
* can't have any temp files (and the arrays might not even exist if
@@ -584,37 +963,13 @@ ExecHashTableDestroy(HashJoinTable hashtable)
pfree(hashtable);
}
-/*
- * ExecHashIncreaseNumBatches
- * increase the original number of batches in order to reduce
- * current memory consumption
- */
static void
-ExecHashIncreaseNumBatches(HashJoinTable hashtable)
+extend_batch_file_arrays(HashJoinTable hashtable, int nbatch)
{
- int oldnbatch = hashtable->nbatch;
- int curbatch = hashtable->curbatch;
- int nbatch;
MemoryContext oldcxt;
- long ninmemory;
- long nfreed;
- HashMemoryChunk oldchunks;
+ int oldnbatch = hashtable->nbatch;
- /* do nothing if we've decided to shut off growth */
- if (!hashtable->growEnabled)
- return;
-
- /* safety check to avoid overflow */
- if (oldnbatch > Min(INT_MAX / 2, MaxAllocSize / (sizeof(void *) * 2)))
- return;
-
- nbatch = oldnbatch * 2;
- Assert(nbatch > 1);
-
-#ifdef HJDEBUG
- printf("Hashjoin %p: increasing nbatch to %d because space = %zu\n",
- hashtable, nbatch, hashtable->spaceUsed);
-#endif
+ TRACE_POSTGRESQL_HASH_INCREASE_BATCHES(nbatch);
oldcxt = MemoryContextSwitchTo(hashtable->hashCxt);
@@ -641,9 +996,49 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
(nbatch - oldnbatch) * sizeof(BufFile *));
}
+ hashtable->nbatch = nbatch;
+
MemoryContextSwitchTo(oldcxt);
+}
- hashtable->nbatch = nbatch;
+/*
+ * ExecHashIncreaseNumBatches
+ * increase the original number of batches in order to reduce
+ * current memory consumption
+ */
+static void
+ExecHashIncreaseNumBatches(HashJoinTable hashtable)
+{
+ int oldnbatch = hashtable->nbatch;
+ int curbatch = hashtable->curbatch;
+ int nbatch;
+ long ninmemory;
+ long nfreed;
+ HashMemoryChunk oldchunks;
+
+ /*
+ * TODO: Should private hash tables also switch to chunk-based memory
+ * accounting, done in dense_alloc, and use ExecHashShrink?
+ */
+ Assert(!HashJoinTableIsShared(hashtable));
+
+ /* do nothing if we've decided to shut off growth */
+ if (!hashtable->growEnabled)
+ return;
+
+ /* safety check to avoid overflow */
+ if (oldnbatch > Min(INT_MAX / 2, MaxAllocSize / (sizeof(void *) * 2)))
+ return;
+
+ nbatch = oldnbatch * 2;
+ Assert(nbatch > 1);
+
+#ifdef HJDEBUG
+ printf("Hashjoin %p: increasing nbatch to %d because space = %zu\n",
+ hashtable, nbatch, hashtable->spaceUsed);
+#endif
+
+ extend_batch_file_arrays(hashtable, nbatch);
/*
* Scan through the existing hash table entries and dump out any that are
@@ -661,7 +1056,7 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
hashtable->log2_nbuckets = hashtable->log2_nbuckets_optimal;
hashtable->buckets = repalloc(hashtable->buckets,
- sizeof(HashJoinTuple) * hashtable->nbuckets);
+ sizeof(HashJoinBucketHead) * hashtable->nbuckets);
}
/*
@@ -669,14 +1064,14 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
* buckets now and not have to keep track which tuples in the buckets have
* already been processed. We will free the old chunks as we go.
*/
- memset(hashtable->buckets, 0, sizeof(HashJoinTuple) * hashtable->nbuckets);
- oldchunks = hashtable->chunks;
- hashtable->chunks = NULL;
+ memset(hashtable->buckets, 0, sizeof(HashJoinBucketHead) * hashtable->nbuckets);
+ oldchunks = hashtable->chunk;
+ hashtable->chunk = NULL;
/* so, let's scan through the old chunks, and all tuples in each chunk */
while (oldchunks != NULL)
{
- HashMemoryChunk nextchunk = oldchunks->next;
+ HashMemoryChunk nextchunk = oldchunks->next.private;
/* position within the buffer (up to oldchunks->used) */
size_t idx = 0;
@@ -699,20 +1094,23 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
/* keep tuple in memory - copy it into the new chunk */
HashJoinTuple copyTuple;
- copyTuple = (HashJoinTuple) dense_alloc(hashtable, hashTupleSize);
+ copyTuple = (HashJoinTuple)
+ dense_alloc(hashtable, hashTupleSize);
memcpy(copyTuple, hashTuple, hashTupleSize);
/* and add it back to the appropriate bucket */
- copyTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = copyTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, copyTuple,
+ InvalidDsaPointer);
}
else
{
/* dump it out */
Assert(batchno > curbatch);
- ExecHashJoinSaveTuple(HJTUPLE_MINTUPLE(hashTuple),
+ ExecHashJoinSaveTuple(hashtable,
+ HJTUPLE_MINTUPLE(hashTuple),
hashTuple->hashvalue,
- &hashtable->innerBatchFile[batchno]);
+ batchno,
+ true);
hashtable->spaceUsed -= hashTupleSize;
nfreed++;
@@ -758,8 +1156,6 @@ ExecHashIncreaseNumBatches(HashJoinTable hashtable)
static void
ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
{
- HashMemoryChunk chunk;
-
/* do nothing if not an increase (it's called increase for a reason) */
if (hashtable->nbuckets >= hashtable->nbuckets_optimal)
return;
@@ -780,45 +1176,412 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
* Just reallocate the proper number of buckets - we don't need to walk
* through them - we can walk the dense-allocated chunks (just like in
* ExecHashIncreaseNumBatches, but without all the copying into new
- * chunks)
+ * chunks): see ExecHashRebucket, which must be called next.
+ */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Size bytes;
+ int i;
+
+ /* Serial phase: only one backend reallocates. */
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_RESIZING);
+
+ /* Free the old hash table. */
+ dsa_free(hashtable->area, hashtable->shared->buckets);
+
+ /* Allocate replacement. */
+ bytes = hashtable->nbuckets * sizeof(HashJoinBucketHead);
+ hashtable->shared->buckets = dsa_allocate(hashtable->area, bytes);
+ if (!DsaPointerIsValid(hashtable->shared->buckets))
+ ereport(ERROR,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("out of memory")));
+
+ /* Initialize empty hash table buckets. */
+ hashtable->buckets =
+ dsa_get_address(hashtable->area,
+ hashtable->shared->buckets);
+ for (i = 0; i < hashtable->nbuckets; ++i)
+ dsa_pointer_atomic_write(&hashtable->buckets[i].shared,
+ InvalidDsaPointer);
+ hashtable->shared->nbuckets = hashtable->nbuckets;
+
+ /* Update size accounting. */
+ hashtable->shared->size += bytes / 2;
+
+ /* Move all chunks to the rebucket list. */
+ hashtable->shared->chunks_to_rebucket = hashtable->shared->chunks;
+ hashtable->shared->chunks = InvalidDsaPointer;
+ }
+ else
+ {
+ hashtable->buckets =
+ (HashJoinBucketHead *) repalloc(hashtable->buckets,
+ hashtable->nbuckets * sizeof(HashJoinBucketHead));
+
+ memset(hashtable->buckets, 0, hashtable->nbuckets * sizeof(HashJoinBucketHead));
+ /* Move all chunks to the rebucket list. */
+ hashtable->chunks_to_rebucket = hashtable->chunk;
+ hashtable->chunk = NULL;
+ }
+}
+
+/*
+ * Pop a memory chunk from a given list. Returns a backend-local pointer to
+ * the chunk, or NULL if the list is empty. Also sets *chunk_out to the
+ * dsa_pointer to the chunk.
+ */
+static HashMemoryChunk
+ExecHashPopChunk(HashJoinTable hashtable,
+ dsa_pointer *chunk_out,
+ dsa_pointer *head)
+{
+ HashMemoryChunk chunk;
+
+ Assert(LWLockHeldByMe(&hashtable->shared->chunk_lock));
+
+ if (!DsaPointerIsValid(*head))
+ return NULL;
+
+ *chunk_out = *head;
+ chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, *chunk_out);
+ *head = chunk->next.shared;
+
+ return chunk;
+}
+
+/*
+ * Push a shared memory chunk onto a given list.
+ */
+static void
+ExecHashPushChunk(HashJoinTable hashtable,
+ HashMemoryChunk chunk,
+ dsa_pointer chunk_shared,
+ dsa_pointer *head)
+{
+ Assert(LWLockHeldByMeInMode(&hashtable->shared->chunk_lock, LW_EXCLUSIVE));
+ Assert(chunk == dsa_get_address(hashtable->area, chunk_shared));
+
+ chunk->next.shared = *head;
+ *head = chunk_shared;
+}
+
+/*
+ * ExecHashRebucket
+ * insert the tuples from hashtable->chunks_to_rebucket into the hashtable
+ */
+void
+ExecHashRebucket(HashJoinTable hashtable)
+{
+ HashMemoryChunk chunk;
+ dsa_pointer chunk_shared;
+ int chunks_processed = 0;
+
+ TRACE_POSTGRESQL_HASH_REBUCKET_START();
+
+ /*
+ * Scan through all tuples in all chunks in the rebucket list to rebuild
+ * the hash table.
+ */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ LWLockAcquire(&hashtable->shared->chunk_lock, LW_EXCLUSIVE);
+ chunk =
+ ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_to_rebucket);
+ LWLockRelease(&hashtable->shared->chunk_lock);
+ }
+ else
+ chunk = hashtable->chunks_to_rebucket;
+ while (chunk != NULL)
+ {
+ /* process all tuples stored in this chunk */
+ size_t idx = 0;
+
+ while (idx < chunk->used)
+ {
+ HashJoinTuple hashTuple = (HashJoinTuple) (chunk->data + idx);
+ dsa_pointer hashTuple_shared = chunk_shared +
+ offsetof(HashMemoryChunkData, data) + idx;
+ int bucketno;
+ int batchno;
+
+ ExecHashGetBucketAndBatch(hashtable, hashTuple->hashvalue,
+ &bucketno, &batchno);
+
+ /* add the tuple to the proper bucket */
+ insert_tuple_into_bucket(hashtable, bucketno, hashTuple,
+ hashTuple_shared);
+
+ /* advance index past the tuple */
+ idx += MAXALIGN(HJTUPLE_OVERHEAD +
+ HJTUPLE_MINTUPLE(hashTuple)->t_len);
+ }
+ ++chunks_processed;
+
+ /* Push chunk back onto the chunk list and move to the next. */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ LWLockAcquire(&hashtable->shared->chunk_lock, LW_EXCLUSIVE);
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ &hashtable->shared->chunks);
+ chunk =
+ ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_to_rebucket);
+ LWLockRelease(&hashtable->shared->chunk_lock);
+ }
+ else
+ {
+ HashMemoryChunk next = chunk->next.private;
+
+ chunk->next.private = hashtable->chunk;
+ hashtable->chunk = chunk;
+ chunk = next;
+ }
+ }
+
+ TRACE_POSTGRESQL_HASH_REBUCKET_DONE(chunks_processed);
+}
+
+static void
+ExecHashTableComputeOptimalBuckets(HashJoinTable hashtable)
+{
+ double ntuples = (hashtable->totalTuples - hashtable->skewTuples);
+
+ /*
+ * Guard against integer overflow and alloc size overflow. The
+ * MaxAllocSize limitation doesn't really apply for shared hash tables,
+ * since DSA has no such limit, but for now let's apply the same limit.
*/
- hashtable->buckets =
- (HashJoinTuple *) repalloc(hashtable->buckets,
- hashtable->nbuckets * sizeof(HashJoinTuple));
+ while (ntuples > (hashtable->nbuckets_optimal * NTUP_PER_BUCKET) &&
+ hashtable->nbuckets_optimal <= INT_MAX / 2 &&
+ hashtable->nbuckets_optimal * 2 <= MaxAllocSize / sizeof(HashJoinBucketHead))
+ {
+ hashtable->nbuckets_optimal *= 2;
+ hashtable->log2_nbuckets_optimal += 1;
+ }
+}
+
+/*
+ * Process the queue of chunks whose tuples need to be redistributed into the
+ * correct batches until it is empty. Hopefully this will shrink the hash
+ * table, keeping about half of the tuples in memory and sending the rest to a
+ * future batch.
+ */
+static void
+ExecHashShrink(HashJoinTable hashtable)
+{
+ Size size_before_shrink = 0;
+ Size tuples_in_memory = 0;
+ Size tuples_written_out = 0;
+ dsa_pointer chunk_shared;
+ HashMemoryChunk chunk;
+ bool elected_to_decide = false;
+
+ TRACE_POSTGRESQL_HASH_SHRINK_START(hashtable->nbatch);
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Since a newly launched participant could arrive while shrinking is
+ * already underway, we need to be able to jump to the correct place
+ * in this function.
+ */
+ switch (BarrierPhase(&hashtable->shared->shrink_barrier))
+ {
+ case PHJ_SHRINK_PHASE_BEGINNING: /* likely case */
+ break;
+ case PHJ_SHRINK_PHASE_CLEARING:
+ goto clearing;
+ case PHJ_SHRINK_PHASE_WORKING:
+ goto working;
+ case PHJ_SHRINK_PHASE_DECIDING:
+ goto deciding;
+ }
+
+ /*
+ * We wait until all participants have reached this point. We need to
+ * do that because we can't clear the hash table if any partipicant is
+ * still inserting tuples into it, and we can't modify chunks that any
+ * participant is still writing into.
+ */
+ if (BarrierWait(&hashtable->shared->shrink_barrier,
+ WAIT_EVENT_HASH_SHRINKING1))
+ {
+ /* TODO: could also resize hash table here! */
+
+ /* Serial phase: one participant clears the hash table. */
+ memset(hashtable->buckets, 0,
+ hashtable->nbuckets * sizeof(HashJoinBucketHead));
- memset(hashtable->buckets, 0, hashtable->nbuckets * sizeof(HashJoinTuple));
+ /*
+ * This participant will also make the decision about whether to
+ * disable further attempts to shrink.
+ */
+ size_before_shrink = hashtable->shared->size;
+ elected_to_decide = true;
+ }
+ clearing:
+ /* Wait until hash table is cleared. */
+ BarrierWait(&hashtable->shared->shrink_barrier,
+ WAIT_EVENT_HASH_SHRINKING2);
+
+ Assert(hashtable->shared->nbatch == hashtable->nbatch);
+ }
+ else
+ {
+ /* Clear the hash table. */
+ memset(hashtable->buckets, 0,
+ sizeof(HashJoinBucketHead) * hashtable->nbuckets);
+ }
+
+ /* Pop first chunk from the shrink queue. */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ working:
+ LWLockAcquire(&hashtable->shared->chunk_lock, LW_EXCLUSIVE);
+ chunk = ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_to_shrink);
+ LWLockRelease(&hashtable->shared->chunk_lock);
+ }
+ else
+ chunk = hashtable->chunks_to_shrink;
+
+ /* Process queue until empty. */
+ while (chunk != NULL)
+ {
+ Size idx = 0;
+
+ /* Process all tuples stored in this chunk. */
+ while (idx < chunk->used)
+ {
+ HashJoinTuple hashTuple = (HashJoinTuple) (chunk->data + idx);
+ MinimalTuple tuple = HJTUPLE_MINTUPLE(hashTuple);
+ dsa_pointer copyTupleShared = InvalidDsaPointer;
+ int hashTupleSize = (HJTUPLE_OVERHEAD + tuple->t_len);
+ int bucketno;
+ int batchno;
+
+ ExecHashGetBucketAndBatch(hashtable, hashTuple->hashvalue,
+ &bucketno, &batchno);
+
+ if (batchno == hashtable->curbatch)
+ {
+ /* keep tuple in memory - copy it into the new chunk */
+ HashJoinTuple copyTuple;
+
+ if (HashJoinTableIsShared(hashtable))
+ copyTuple = (HashJoinTuple)
+ dense_alloc_shared(hashtable, hashTupleSize,
+ ©TupleShared, false, false);
+ else
+ copyTuple = (HashJoinTuple)
+ dense_alloc(hashtable, hashTupleSize);
+ memcpy(copyTuple, hashTuple, hashTupleSize);
+
+ /* and add it back to the appropriate bucket */
+ insert_tuple_into_bucket(hashtable, bucketno, copyTuple,
+ copyTupleShared);
+ ++tuples_in_memory;
+ }
+ else
+ {
+ /* dump it out */
+ Assert(batchno > hashtable->curbatch);
+ ExecHashJoinSaveTuple(hashtable,
+ HJTUPLE_MINTUPLE(hashTuple),
+ hashTuple->hashvalue,
+ batchno,
+ true);
+
+ hashtable->spaceUsed -= hashTupleSize;
+ ++tuples_written_out;
+ }
- /* scan through all tuples in all chunks to rebuild the hash table */
- for (chunk = hashtable->chunks; chunk != NULL; chunk = chunk->next)
- {
- /* process all tuples stored in this chunk */
- size_t idx = 0;
+ /* next tuple in this chunk */
+ idx += MAXALIGN(hashTupleSize);
+ }
- while (idx < chunk->used)
+ /* Free chunk and pop next from the shrink queue. */
+ if (HashJoinTableIsShared(hashtable))
{
- HashJoinTuple hashTuple = (HashJoinTuple) (chunk->data + idx);
- int bucketno;
- int batchno;
+ Size size = chunk->maxlen + offsetof(HashMemoryChunkData, data);
+
+ TRACE_POSTGRESQL_HASH_FREE_CHUNK(size);
+ dsa_free(hashtable->area, chunk_shared);
+
+ LWLockAcquire(&hashtable->shared->chunk_lock, LW_EXCLUSIVE);
+ Assert(hashtable->shared->size > size);
+ hashtable->shared->size -= size;
+ hashtable->shared->tuples_in_memory += tuples_in_memory;
+ hashtable->shared->tuples_written_out += tuples_written_out;
+ tuples_in_memory = 0;
+ tuples_written_out = 0;
+ chunk = ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_to_shrink);
+ LWLockRelease(&hashtable->shared->chunk_lock);
+ }
+ else
+ {
+ HashMemoryChunk next = chunk->next.private;
- ExecHashGetBucketAndBatch(hashtable, hashTuple->hashvalue,
- &bucketno, &batchno);
+ pfree(chunk);
+ chunk = next;
+ }
+ }
- /* add the tuple to the proper bucket */
- hashTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = hashTuple;
+ /* Decide if shrinking actually reduced memory usage. */
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Wait until all have finished shrinking chunks. We need to do that
+ * because we need the total tuple counts before we can decide whether
+ * to prevent further attempts at shrinking.
+ */
+ BarrierWait(&hashtable->shared->shrink_barrier,
+ WAIT_EVENT_HASH_SHRINKING3);
- /* advance index past the tuple */
- idx += MAXALIGN(HJTUPLE_OVERHEAD +
- HJTUPLE_MINTUPLE(hashTuple)->t_len);
+ if (elected_to_decide)
+ {
+ /* Serial phase: one participant decides. */
+ if (hashtable->shared->tuples_in_memory == 0 ||
+ hashtable->shared->tuples_written_out == 0)
+ {
+ TRACE_POSTGRESQL_HASH_SHRINK_DISABLED();
+ hashtable->shared->shrinking_enabled = false;
+ }
+
+ TRACE_POSTGRESQL_HASH_SHRINK_STATS(hashtable->shared->tuples_in_memory,
+ hashtable->shared->tuples_written_out,
+ size_before_shrink,
+ hashtable->shared->size);
+ }
+ deciding:
+ /* Wait for above decision to be made. */
+ BarrierWaitSet(&hashtable->shared->shrink_barrier,
+ PHJ_SHRINK_PHASE_BEGINNING,
+ WAIT_EVENT_HASH_SHRINKING4);
+ }
+ else
+ {
+ if (tuples_in_memory == 0 || tuples_written_out == 0)
+ {
+ TRACE_POSTGRESQL_HASH_SHRINK_DISABLED();
+ hashtable->growEnabled = false;
}
}
-}
+ TRACE_POSTGRESQL_HASH_SHRINK_DONE();
+}
/*
* ExecHashTableInsert
* insert a tuple into the hash table depending on the hash value
- * it may just go to a temp file for later batches
+ * it may just go to a temp file for later batches; if 'preload' is
+ * then it may be loaded into a chunk but not actually inserted yet;
+ * return true on success, false if we ran out of work_mem
*
* Note: the passed TupleTableSlot may contain a regular, minimal, or virtual
* tuple; the minimal case in particular is certain to happen while reloading
@@ -826,10 +1589,11 @@ ExecHashIncreaseNumBuckets(HashJoinTable hashtable)
* case by not forcing the slot contents into minimal form; not clear if it's
* worth the messiness required.
*/
-void
+bool
ExecHashTableInsert(HashJoinTable hashtable,
TupleTableSlot *slot,
- uint32 hashvalue)
+ uint32 hashvalue,
+ bool preload)
{
MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot);
int bucketno;
@@ -839,20 +1603,61 @@ ExecHashTableInsert(HashJoinTable hashtable,
&bucketno, &batchno);
/*
- * decide whether to put the tuple in the hash table or a temp file
+ * decide whether to put the tuple in memory or in a temp file
*/
- if (batchno == hashtable->curbatch)
+ if (batchno == hashtable->curbatch + (preload ? 1 : 0))
{
/*
* put the tuple in hash table
*/
HashJoinTuple hashTuple;
int hashTupleSize;
- double ntuples = (hashtable->totalTuples - hashtable->skewTuples);
+ dsa_pointer hashTuple_shared = InvalidDsaPointer;
/* Create the HashJoinTuple */
hashTupleSize = HJTUPLE_OVERHEAD + tuple->t_len;
- hashTuple = (HashJoinTuple) dense_alloc(hashtable, hashTupleSize);
+
+ retry:
+ if (HashJoinTableIsShared(hashtable))
+ hashTuple = (HashJoinTuple)
+ dense_alloc_shared(hashtable, hashTupleSize,
+ &hashTuple_shared, preload, true);
+ else
+ hashTuple = (HashJoinTuple)
+ dense_alloc(hashtable, hashTupleSize);
+
+ /* Check for failure for allocate. */
+ if (!hashTuple)
+ {
+ if (preload)
+ {
+ /*
+ * There is no more work_mem into which to preload tuples for
+ * the next batch, so tell caller to stop doing that.
+ */
+ Assert(HashJoinTableIsShared(hashtable));
+ return false;
+ }
+ else
+ {
+ /*
+ * Either dense_alloc_shared has decided that we should
+ * increase the number of batches or another participant has
+ * already decided to do that, so we should go and help shrink
+ * the hash table by sending tuples to future batches.
+ */
+ Assert(HashJoinTableIsShared(hashtable));
+ ExecHashShrink(hashtable);
+
+ /*
+ * Try again. Hopefully memory has been freed up, or we've
+ * decided to stop respecting work_mem because increasing the
+ * number of batches isn't helping (large numbers of tuples
+ * with the same hash value can't be separated).
+ */
+ goto retry;
+ }
+ }
hashTuple->hashvalue = hashvalue;
memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
@@ -865,33 +1670,32 @@ ExecHashTableInsert(HashJoinTable hashtable,
*/
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple));
- /* Push it onto the front of the bucket's list */
- hashTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = hashTuple;
+ /* Push it onto the front of the bucket's list, unless preloading */
+ if (!preload)
+ insert_tuple_into_bucket(hashtable, bucketno, hashTuple,
+ hashTuple_shared);
/*
* Increase the (optimal) number of buckets if we just exceeded the
* NTUP_PER_BUCKET threshold, but only when there's still a single
* batch.
*/
- if (hashtable->nbatch == 1 &&
- ntuples > (hashtable->nbuckets_optimal * NTUP_PER_BUCKET))
- {
- /* Guard against integer overflow and alloc size overflow */
- if (hashtable->nbuckets_optimal <= INT_MAX / 2 &&
- hashtable->nbuckets_optimal * 2 <= MaxAllocSize / sizeof(HashJoinTuple))
- {
- hashtable->nbuckets_optimal *= 2;
- hashtable->log2_nbuckets_optimal += 1;
- }
- }
+ if (hashtable->nbatch == 1)
+ ExecHashTableComputeOptimalBuckets(hashtable);
+
+ /*
+ * TODO: Get rid of the following code, and use the same pattern as
+ * above, namely let dense_alloc count chunk size (it's more
+ * accurate!) and let it tell you when you need to back off and
+ * ExecHashShrink?
+ */
/* Account for space used, and back off if we've used too much */
hashtable->spaceUsed += hashTupleSize;
if (hashtable->spaceUsed > hashtable->spacePeak)
hashtable->spacePeak = hashtable->spaceUsed;
if (hashtable->spaceUsed +
- hashtable->nbuckets_optimal * sizeof(HashJoinTuple)
+ hashtable->nbuckets_optimal * sizeof(HashJoinBucketHead)
> hashtable->spaceAllowed)
ExecHashIncreaseNumBatches(hashtable);
}
@@ -900,11 +1704,15 @@ ExecHashTableInsert(HashJoinTable hashtable,
/*
* put the tuple into a temp file for later batches
*/
- Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(tuple,
+ Assert(batchno > hashtable->curbatch + (preload ? 1 : 0));
+ ExecHashJoinSaveTuple(hashtable,
+ tuple,
hashvalue,
- &hashtable->innerBatchFile[batchno]);
+ batchno,
+ true);
}
+
+ return true;
}
/*
@@ -1047,6 +1855,134 @@ ExecHashGetBucketAndBatch(HashJoinTable hashtable,
}
/*
+ * Update the local hashtable with the current pointers and sizes from
+ * hashtable->parallel_state.
+ */
+void
+ExecHashUpdate(HashJoinTable hashtable)
+{
+ Barrier *barrier;
+
+ if (!HashJoinTableIsShared(hashtable))
+ return;
+
+ barrier = &hashtable->shared->barrier;
+
+ /*
+ * This should only be called in a phase when the hash table is not being
+ * mutated (ie resized, swapped etc).
+ */
+ Assert(!PHJ_PHASE_MUTATING_TABLE(
+ BarrierPhase(&hashtable->shared->barrier)));
+
+ /* The hash table. */
+ hashtable->buckets = (HashJoinBucketHead *)
+ dsa_get_address(hashtable->area, hashtable->shared->buckets);
+ hashtable->nbuckets = hashtable->shared->nbuckets;
+ /* TODO nbatch? */
+ hashtable->log2_nbuckets = my_log2(hashtable->nbuckets);
+
+ hashtable->curbatch = PHJ_PHASE_TO_BATCHNO(BarrierPhase(barrier));
+}
+
+/*
+ * Get the next tuple in the same bucket as 'tuple'.
+ */
+static HashJoinTuple
+next_tuple_in_bucket(HashJoinTable table, HashJoinTuple tuple)
+{
+ if (HashJoinTableIsShared(table))
+ return (HashJoinTuple)
+ dsa_get_address(table->area, tuple->next.shared);
+ else
+ return tuple->next.private;
+}
+
+/*
+ * Get the first tuple in a given skew bucket identified by number.
+ */
+static HashJoinTuple
+first_tuple_in_skew_bucket(HashJoinTable table, int skew_bucket_no)
+{
+ if (HashJoinTableIsShared(table))
+ return (HashJoinTuple)
+ dsa_get_address(table->area,
+ table->skewBucket[skew_bucket_no]->tuples.shared);
+ else
+ return table->skewBucket[skew_bucket_no]->tuples.private;
+}
+
+/*
+ * Get the first tuple in a given bucket identified by number.
+ */
+static HashJoinTuple
+first_tuple_in_bucket(HashJoinTable table, int bucket_no)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ dsa_pointer p =
+ dsa_pointer_atomic_read(&table->buckets[bucket_no].shared);
+ return (HashJoinTuple) dsa_get_address(table->area, p);
+ }
+ else
+ return table->buckets[bucket_no].private;
+}
+
+/*
+ * Insert a tuple at the front of a given bucket identified by number. For
+ * shared hash joins, tuple_shared must be provided, pointing to the tuple in
+ * the dsa_area backing the table. For private hash joins, it should be
+ * InvalidDsaPointer.
+ */
+static void
+insert_tuple_into_bucket(HashJoinTable table, int bucket_no,
+ HashJoinTuple tuple, dsa_pointer tuple_shared)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ Assert(tuple == dsa_get_address(table->area, tuple_shared));
+ for (;;)
+ {
+ tuple->next.shared =
+ dsa_pointer_atomic_read(&table->buckets[bucket_no].shared);
+ if (dsa_pointer_atomic_compare_exchange(&table->buckets[bucket_no].shared,
+ &tuple->next.shared,
+ tuple_shared))
+ break;
+ }
+ }
+ else
+ {
+ tuple->next.private = table->buckets[bucket_no].private;
+ table->buckets[bucket_no].private = tuple;
+ }
+}
+
+/*
+ * Insert a tuple at the front of a given skew bucket identified by number.
+ * For shared hash joins, tuple_shared must be provided, pointing to the tuple
+ * in the dsa_area backing the table. For private hash joins, it should be
+ * InvalidDsaPointer.
+ */
+static void
+insert_tuple_into_skew_bucket(HashJoinTable table, int skew_bucket_no,
+ HashJoinTuple tuple,
+ dsa_pointer tuple_shared)
+{
+ if (HashJoinTableIsShared(table))
+ {
+ tuple->next.shared =
+ table->skewBucket[skew_bucket_no]->tuples.shared;
+ table->skewBucket[skew_bucket_no]->tuples.shared = tuple_shared;
+ }
+ else
+ {
+ tuple->next.private = table->skewBucket[skew_bucket_no]->tuples.private;
+ table->skewBucket[skew_bucket_no]->tuples.private = tuple;
+ }
+}
+
+/*
* ExecScanHashBucket
* scan a hash bucket for matches to the current outer tuple
*
@@ -1073,11 +2009,12 @@ ExecScanHashBucket(HashJoinState *hjstate,
* otherwise scan the standard hashtable bucket.
*/
if (hashTuple != NULL)
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
else if (hjstate->hj_CurSkewBucketNo != INVALID_SKEW_BUCKET_NO)
- hashTuple = hashtable->skewBucket[hjstate->hj_CurSkewBucketNo]->tuples;
+ hashTuple = first_tuple_in_skew_bucket(hashtable,
+ hjstate->hj_CurSkewBucketNo);
else
- hashTuple = hashtable->buckets[hjstate->hj_CurBucketNo];
+ hashTuple = first_tuple_in_bucket(hashtable, hjstate->hj_CurBucketNo);
while (hashTuple != NULL)
{
@@ -1101,7 +2038,7 @@ ExecScanHashBucket(HashJoinState *hjstate,
}
}
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
}
/*
@@ -1144,6 +2081,81 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
HashJoinTable hashtable = hjstate->hj_HashTable;
HashJoinTuple hashTuple = hjstate->hj_CurTuple;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_UNMATCHED_BATCH(hashtable->curbatch));
+
+ /*
+ * For the parallel verison, we'll let each participant pull chunks
+ * from the queue to work on independently.
+ */
+ for (;;)
+ {
+ /* Do we need a new chunk? */
+ if (hashtable->chunk == NULL)
+ {
+ dsa_pointer chunk_shared;
+
+ /*
+ * Try to pop a chunk from the unmatched queue, and put it
+ * back on the main chunks list.
+ */
+ LWLockAcquire(&hashtable->shared->chunk_lock, LW_EXCLUSIVE);
+ hashtable->chunk =
+ ExecHashPopChunk(hashtable, &chunk_shared,
+ &hashtable->shared->chunks_unmatched);
+ if (hashtable->chunk != NULL)
+ ExecHashPushChunk(hashtable, hashtable->chunk,
+ chunk_shared,
+ &hashtable->shared->chunks);
+ LWLockRelease(&hashtable->shared->chunk_lock);
+
+ /* If no more chunks in the queue: we're done. */
+ if (hashtable->chunk == NULL)
+ return false;
+
+ hashtable->chunk_unmatched_pos = 0;
+ }
+
+ /* Does the current chunk have any more tuples? */
+ if (hashtable->chunk_unmatched_pos >= hashtable->chunk->used)
+ {
+ /* Try a new chunk. */
+ hashtable->chunk = NULL;
+ continue;
+ }
+ hashTuple = (HashJoinTuple)
+ hashtable->chunk->data + hashtable->chunk_unmatched_pos;
+
+ /* Move to the next tuple in this chunk. */
+ hashtable->chunk_unmatched_pos +=
+ HJTUPLE_OVERHEAD + HJTUPLE_MINTUPLE(hashTuple)->t_len;
+
+ /* Is it unmatched? */
+ if (!HeapTupleHeaderHasMatch(HJTUPLE_MINTUPLE(hashTuple)))
+ {
+ TupleTableSlot *inntuple;
+
+ /* insert hashtable's tuple into exec slot */
+ inntuple = ExecStoreMinimalTuple(HJTUPLE_MINTUPLE(hashTuple),
+ hjstate->hj_HashTupleSlot,
+ false); /* do not pfree */
+ econtext->ecxt_innertuple = inntuple;
+
+ /*
+ * Reset temp memory each time; although this function doesn't
+ * do any qual eval, the caller will, so let's keep it
+ * parallel to ExecScanHashBucket.
+ */
+ ResetExprContext(econtext);
+
+ hjstate->hj_CurTuple = hashTuple;
+ return true;
+ }
+ }
+ }
+
for (;;)
{
/*
@@ -1152,21 +2164,21 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
* bucket.
*/
if (hashTuple != NULL)
- hashTuple = hashTuple->next;
- else if (hjstate->hj_CurBucketNo < hashtable->nbuckets)
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
{
- hashTuple = hashtable->buckets[hjstate->hj_CurBucketNo];
- hjstate->hj_CurBucketNo++;
- }
- else if (hjstate->hj_CurSkewBucketNo < hashtable->nSkewBuckets)
- {
- int j = hashtable->skewBucketNums[hjstate->hj_CurSkewBucketNo];
+ if (hjstate->hj_CurBucketNo < hashtable->nbuckets)
+ hashTuple = first_tuple_in_bucket(hashtable,
+ hjstate->hj_CurBucketNo++);
+ else if (hjstate->hj_CurSkewBucketNo < hashtable->nSkewBuckets)
+ {
+ int j = hashtable->skewBucketNums[hjstate->hj_CurSkewBucketNo];
- hashTuple = hashtable->skewBucket[j]->tuples;
- hjstate->hj_CurSkewBucketNo++;
+ hashTuple = first_tuple_in_skew_bucket(hashtable, j);
+ hjstate->hj_CurSkewBucketNo++;
+ }
+ else
+ break; /* finished all buckets */
}
- else
- break; /* finished all buckets */
while (hashTuple != NULL)
{
@@ -1191,7 +2203,7 @@ ExecScanHashTableForUnmatched(HashJoinState *hjstate, ExprContext *econtext)
return true;
}
- hashTuple = hashTuple->next;
+ hashTuple = next_tuple_in_bucket(hashtable, hashTuple);
}
}
@@ -1212,6 +2224,59 @@ ExecHashTableReset(HashJoinTable hashtable)
MemoryContext oldcxt;
int nbuckets = hashtable->nbuckets;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* Wait for all workers to finish accessing the hash table. */
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_UNMATCHED);
+ if (BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASH_UNMATCHED))
+ {
+ /* Serial phase: set up hash table for new batch. */
+ int i;
+
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_PROMOTING);
+
+ /* Clear the hash table. */
+ for (i = 0; i < nbuckets; ++i)
+ dsa_pointer_atomic_write(&hashtable->buckets[i].shared,
+ InvalidDsaPointer);
+
+ /* Free all the chunks. */
+ /* TODO: Put them on a freelist instead? Better than making one backend free them all! */
+ while (DsaPointerIsValid(hashtable->shared->chunks))
+ {
+ HashMemoryChunk chunk = (HashMemoryChunk)
+ dsa_get_address(hashtable->area, hashtable->shared->chunks);
+ dsa_pointer next = chunk->next.shared;
+
+ dsa_free(hashtable->area, hashtable->shared->chunks);
+ hashtable->shared->chunks = next;
+ }
+
+ /* Any preloaded chunks for the next batch need to be bucketed. */
+ hashtable->shared->chunks_to_rebucket =
+ hashtable->shared->chunks_preloaded;
+ hashtable->shared->chunks_preloaded = InvalidDsaPointer;
+
+ /* Update the hash table size: it now has the preloaded chunks. */
+ hashtable->shared->size =
+ (hashtable->nbuckets * sizeof(HashJoinBucketHead)) +
+ hashtable->shared->size_preloaded;
+ hashtable->shared->size_preloaded = 0;
+ }
+ /* Wait again, so that all workers now have the new table. */
+ BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASH_PROMOTING);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_LOADING);
+ ExecHashUpdate(hashtable);
+
+ /* Forget the current chunks. */
+ hashtable->chunk = NULL;
+ hashtable->chunk_preload = NULL;
+ return;
+ }
+
/*
* Release all the hash buckets and tuples acquired in the prior pass, and
* reinitialize the context for a new pass.
@@ -1220,15 +2285,15 @@ ExecHashTableReset(HashJoinTable hashtable)
oldcxt = MemoryContextSwitchTo(hashtable->batchCxt);
/* Reallocate and reinitialize the hash bucket headers. */
- hashtable->buckets = (HashJoinTuple *)
- palloc0(nbuckets * sizeof(HashJoinTuple));
+ hashtable->buckets = (HashJoinBucketHead *)
+ palloc0(nbuckets * sizeof(HashJoinBucketHead));
hashtable->spaceUsed = 0;
MemoryContextSwitchTo(oldcxt);
/* Forget the chunks (the memory was freed by the context reset above). */
- hashtable->chunks = NULL;
+ hashtable->chunk = NULL;
}
/*
@@ -1241,10 +2306,14 @@ ExecHashTableResetMatchFlags(HashJoinTable hashtable)
HashJoinTuple tuple;
int i;
+ /* TODO: share this work out? */
+
/* Reset all flags in the main table ... */
for (i = 0; i < hashtable->nbuckets; i++)
{
- for (tuple = hashtable->buckets[i]; tuple != NULL; tuple = tuple->next)
+ for (tuple = first_tuple_in_bucket(hashtable, i);
+ tuple != NULL;
+ tuple = next_tuple_in_bucket(hashtable, tuple))
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(tuple));
}
@@ -1252,9 +2321,10 @@ ExecHashTableResetMatchFlags(HashJoinTable hashtable)
for (i = 0; i < hashtable->nSkewBuckets; i++)
{
int j = hashtable->skewBucketNums[i];
- HashSkewBucket *skewBucket = hashtable->skewBucket[j];
- for (tuple = skewBucket->tuples; tuple != NULL; tuple = tuple->next)
+ for (tuple = first_tuple_in_skew_bucket(hashtable, j);
+ tuple != NULL;
+ tuple = next_tuple_in_bucket(hashtable, tuple))
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(tuple));
}
}
@@ -1414,11 +2484,11 @@ ExecHashBuildSkewHash(HashJoinTable hashtable, Hash *node, int mcvsToUse)
continue;
/* Okay, create a new skew bucket for this hashvalue. */
- hashtable->skewBucket[bucket] = (HashSkewBucket *)
+ hashtable->skewBucket[bucket] = (HashSkewBucket *) /* TODO */
MemoryContextAlloc(hashtable->batchCxt,
sizeof(HashSkewBucket));
hashtable->skewBucket[bucket]->hashvalue = hashvalue;
- hashtable->skewBucket[bucket]->tuples = NULL;
+ hashtable->skewBucket[bucket]->tuples.private = NULL;
hashtable->skewBucketNums[hashtable->nSkewBuckets] = bucket;
hashtable->nSkewBuckets++;
hashtable->spaceUsed += SKEW_BUCKET_OVERHEAD;
@@ -1496,18 +2566,29 @@ ExecHashSkewTableInsert(HashJoinTable hashtable,
MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot);
HashJoinTuple hashTuple;
int hashTupleSize;
+ dsa_pointer tuple_pointer;
/* Create the HashJoinTuple */
hashTupleSize = HJTUPLE_OVERHEAD + tuple->t_len;
- hashTuple = (HashJoinTuple) MemoryContextAlloc(hashtable->batchCxt,
- hashTupleSize);
+ if (HashJoinTableIsShared(hashtable))
+ {
+ tuple_pointer = dsa_allocate(hashtable->area, hashTupleSize);
+ hashTuple = (HashJoinTuple) dsa_get_address(hashtable->area,
+ tuple_pointer);
+ }
+ else
+ {
+ tuple_pointer = InvalidDsaPointer;
+ hashTuple = (HashJoinTuple) MemoryContextAlloc(hashtable->batchCxt,
+ hashTupleSize);
+ }
hashTuple->hashvalue = hashvalue;
memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
HeapTupleHeaderClearMatch(HJTUPLE_MINTUPLE(hashTuple));
/* Push it onto the front of the skew bucket's list */
- hashTuple->next = hashtable->skewBucket[bucketNumber]->tuples;
- hashtable->skewBucket[bucketNumber]->tuples = hashTuple;
+ insert_tuple_into_skew_bucket(hashtable, bucketNumber, hashTuple,
+ tuple_pointer);
/* Account for space used, and back off if we've used too much */
hashtable->spaceUsed += hashTupleSize;
@@ -1538,6 +2619,9 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
int batchno;
HashJoinTuple hashTuple;
+ /* TODO: skew buckets not yet supported for parallel mode */
+ Assert(!HashJoinTableIsShared(hashtable));
+
/* Locate the bucket to remove */
bucketToRemove = hashtable->skewBucketNums[hashtable->nSkewBuckets - 1];
bucket = hashtable->skewBucket[bucketToRemove];
@@ -1552,10 +2636,10 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno);
/* Process all tuples in the bucket */
- hashTuple = bucket->tuples;
+ hashTuple = first_tuple_in_skew_bucket(hashtable, bucketToRemove);
while (hashTuple != NULL)
{
- HashJoinTuple nextHashTuple = hashTuple->next;
+ HashJoinTuple nextHashTuple = next_tuple_in_bucket(hashtable, hashTuple);
MinimalTuple tuple;
Size tupleSize;
@@ -1581,8 +2665,8 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
memcpy(copyTuple, hashTuple, tupleSize);
pfree(hashTuple);
- copyTuple->next = hashtable->buckets[bucketno];
- hashtable->buckets[bucketno] = copyTuple;
+ insert_tuple_into_bucket(hashtable, bucketno, copyTuple,
+ InvalidDsaPointer);
/* We have reduced skew space, but overall space doesn't change */
hashtable->spaceUsedSkew -= tupleSize;
@@ -1591,8 +2675,8 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
{
/* Put the tuple into a temp file for later batches */
Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(tuple, hashvalue,
- &hashtable->innerBatchFile[batchno]);
+ ExecHashJoinSaveTuple(hashtable, tuple, hashvalue,
+ batchno, true);
pfree(hashTuple);
hashtable->spaceUsed -= tupleSize;
hashtable->spaceUsedSkew -= tupleSize;
@@ -1636,6 +2720,198 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable)
}
/*
+ * Allocate 'size' bytes from the currently active shared HashMemoryChunk, or
+ * create a new chunk if necessary. This is similar to the private memory
+ * version, but also deals with 'preload' chunks and coordination with other
+ * participants.
+ *
+ * If respect_work_mem is true, then return NULL if the number of batches has
+ * been increased in order to avoid exceeding work_mem. Pass false to allow
+ * work_mem to be exceeded (as can be temporarily needed by ExecHashShrink, or
+ * if increasing the number of batches doesn't seem to be helping us shrink
+ * the memory usage).
+ */
+static void *
+dense_alloc_shared(HashJoinTable hashtable,
+ Size size,
+ dsa_pointer *shared,
+ bool preload,
+ bool respect_work_mem)
+{
+ dsa_pointer chunk_shared;
+ HashMemoryChunk chunk;
+ Size chunk_size;
+
+ /* just in case the size is not already aligned properly */
+ size = MAXALIGN(size);
+
+ /*
+ * Fast path: if there is enough space in this backend's current chunk,
+ * then we can allocate without any locking or work_mem accounting. If
+ * HASH_CHUNK_SIZE is large enough, this strategy should keep lock
+ * contention low. It doesn't matter if another participant has decided
+ * to increase the number of batches; we'll finish filling up this chunk
+ * and then find out about the increase when we need to allocate a new
+ * chunk.
+ */
+ chunk = preload ? hashtable->chunk_preload : hashtable->chunk;
+ if (chunk != NULL &&
+ size < HASH_CHUNK_THRESHOLD &&
+ chunk->maxlen - chunk->used >= size)
+ {
+ void *result;
+
+ chunk_shared = preload
+ ? hashtable->chunk_preload_shared
+ : hashtable->chunk_shared;
+ Assert(chunk == dsa_get_address(hashtable->area, chunk_shared));
+ *shared = chunk_shared +
+ offsetof(HashMemoryChunkData, data) +
+ chunk->used;
+ result = chunk->data + chunk->used;
+ chunk->used += size;
+ chunk->ntuples += 1;
+
+ Assert(chunk->used <= chunk->maxlen);
+ Assert(result == dsa_get_address(hashtable->area, *shared));
+
+ return result;
+ }
+
+ /*
+ * Slow path: try to allocate a new chunk, while also coordinating with
+ * other participants to keep memory usage under work_mem by increasing
+ * the number of batches as required.
+ */
+ LWLockAcquire(&hashtable->shared->chunk_lock, LW_EXCLUSIVE);
+
+ /* Check if some other participant has increased nbatch. */
+ if (hashtable->shared->nbatch > hashtable->nbatch)
+ {
+ Assert(!preload);
+ Assert(respect_work_mem);
+ extend_batch_file_arrays(hashtable, hashtable->shared->nbatch);
+
+ hashtable->chunk = NULL;
+ hashtable->chunk_shared = InvalidDsaPointer;
+ LWLockRelease(&hashtable->shared->chunk_lock);
+
+ /*
+ * Whenever nbatch changes, every participant attached to
+ * shrink_barrier must run ExecHashShrink to help shrink the hash
+ * table. So return NULL to tell caller to go and do that.
+ */
+ return NULL;
+ }
+
+ /* Oversized tuples get their own chunk. */
+ if (size > HASH_CHUNK_THRESHOLD)
+ chunk_size = size + offsetof(HashMemoryChunkData, data);
+ else
+ chunk_size = HASH_CHUNK_SIZE;
+
+ /* If appropriate, check if work_mem would be exceeded by a new chunk. */
+ if (respect_work_mem &&
+ hashtable->shared->shrinking_enabled &&
+ (hashtable->shared->size +
+ hashtable->shared->size_preloaded +
+ chunk_size) > (work_mem * 1024L))
+ {
+ /*
+ * It would. If allocating for the current batch (ie not preloading
+ * the next batch), increase number of batches so we can shrink the
+ * hash table.
+ */
+ if (!preload)
+ {
+ hashtable->shared->nbatch *= 2;
+ extend_batch_file_arrays(hashtable, hashtable->shared->nbatch);
+
+ /* All allocated chunks now need to be shrunk. */
+ hashtable->shared->chunks_to_shrink = hashtable->shared->chunks;
+ hashtable->shared->chunks = InvalidDsaPointer;
+ hashtable->shared->tuples_in_memory = 0;
+ hashtable->shared->tuples_written_out = 0;
+
+ hashtable->chunk = NULL;
+ hashtable->chunk_shared = InvalidDsaPointer;
+ }
+ LWLockRelease(&hashtable->shared->chunk_lock);
+
+ /*
+ * If the caller is preloading, it should now stop doing that because
+ * there is no more work_mem. If it is loading, it should now run
+ * ExecHashShrink so we can get some memory back.
+ */
+ return NULL;
+ }
+
+ /* We are cleared to allocate a new chunk. */
+ chunk_shared = dsa_allocate(hashtable->area, chunk_size);
+ if (!DsaPointerIsValid(chunk_shared))
+ ereport(ERROR,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("out of memory")));
+ TRACE_POSTGRESQL_HASH_ALLOCATE_CHUNK(chunk_size);
+ if (preload)
+ hashtable->shared->size_preloaded += chunk_size;
+ else
+ hashtable->shared->size += chunk_size;
+
+ /* Set up the chunk. */
+ chunk = (HashMemoryChunk) dsa_get_address(hashtable->area, chunk_shared);
+ *shared = chunk_shared + offsetof(HashMemoryChunkData, data);
+ chunk->maxlen = chunk_size - offsetof(HashMemoryChunkData, data);
+ chunk->used = size;
+ chunk->ntuples = 1;
+
+ /*
+ * Push it onto the appropriate list of chunks, so that it can be found if
+ * we need to rebucket or shrink the whole hash table.
+ */
+ ExecHashPushChunk(hashtable, chunk, chunk_shared,
+ preload
+ ? &hashtable->shared->chunks_preloaded
+ : &hashtable->shared->chunks);
+
+ if (size > HASH_CHUNK_THRESHOLD)
+ {
+ /*
+ * Count oversized tuples immediately, but don't bother making this
+ * chunk the 'current' chunk because it has no more space in it for
+ * next time.
+ */
+ if (preload)
+ ++hashtable->shared->tuples_next_batch;
+ else
+ ++hashtable->shared->tuples_this_batch;
+ }
+ else
+ {
+ /*
+ * Make this the current chunk so that we can use the fast path to
+ * fill the rest of it up in future called. We will count this tuple
+ * later, when the chunk is full.
+ */
+ if (preload)
+ {
+ hashtable->chunk_preload = chunk;
+ hashtable->chunk_preload_shared = chunk_shared;
+ }
+ else
+ {
+ hashtable->chunk = chunk;
+ hashtable->chunk_shared = chunk_shared;
+ }
+ }
+ LWLockRelease(&hashtable->shared->chunk_lock);
+
+ Assert(chunk->data == dsa_get_address(hashtable->area, *shared));
+
+ return chunk->data;
+}
+
+/*
* Allocate 'size' bytes from the currently active HashMemoryChunk
*/
static void *
@@ -1653,26 +2929,28 @@ dense_alloc(HashJoinTable hashtable, Size size)
*/
if (size > HASH_CHUNK_THRESHOLD)
{
+
/* allocate new chunk and put it at the beginning of the list */
- newChunk = (HashMemoryChunk) MemoryContextAlloc(hashtable->batchCxt,
- offsetof(HashMemoryChunkData, data) + size);
+ newChunk = (HashMemoryChunk)
+ MemoryContextAlloc(hashtable->batchCxt,
+ offsetof(HashMemoryChunkData, data) + size);
newChunk->maxlen = size;
newChunk->used = 0;
- newChunk->ntuples = 0;
+ newChunk->ntuples= 0;
/*
* Add this chunk to the list after the first existing chunk, so that
* we don't lose the remaining space in the "current" chunk.
*/
- if (hashtable->chunks != NULL)
+ if (hashtable->chunk != NULL)
{
- newChunk->next = hashtable->chunks->next;
- hashtable->chunks->next = newChunk;
+ newChunk->next.private = hashtable->chunk->next.private;
+ hashtable->chunk->next.private = newChunk;
}
else
{
- newChunk->next = hashtable->chunks;
- hashtable->chunks = newChunk;
+ newChunk->next.private = NULL;
+ hashtable->chunk = newChunk;
}
newChunk->used += size;
@@ -1685,27 +2963,27 @@ dense_alloc(HashJoinTable hashtable, Size size)
* See if we have enough space for it in the current chunk (if any). If
* not, allocate a fresh chunk.
*/
- if ((hashtable->chunks == NULL) ||
- (hashtable->chunks->maxlen - hashtable->chunks->used) < size)
+ if ((hashtable->chunk == NULL) ||
+ (hashtable->chunk->maxlen - hashtable->chunk->used) < size)
{
/* allocate new chunk and put it at the beginning of the list */
- newChunk = (HashMemoryChunk) MemoryContextAlloc(hashtable->batchCxt,
- offsetof(HashMemoryChunkData, data) + HASH_CHUNK_SIZE);
-
+ newChunk = (HashMemoryChunk)
+ MemoryContextAlloc(hashtable->batchCxt,
+ offsetof(HashMemoryChunkData, data) +
+ HASH_CHUNK_SIZE);
+ newChunk->next.private = hashtable->chunk;
+ hashtable->chunk = newChunk;
newChunk->maxlen = HASH_CHUNK_SIZE;
newChunk->used = size;
newChunk->ntuples = 1;
- newChunk->next = hashtable->chunks;
- hashtable->chunks = newChunk;
-
return newChunk->data;
}
/* There is enough space in the current chunk, let's add the tuple */
- ptr = hashtable->chunks->data + hashtable->chunks->used;
- hashtable->chunks->used += size;
- hashtable->chunks->ntuples += 1;
+ ptr = hashtable->chunk->data + hashtable->chunk->used;
+ hashtable->chunk->used += size;
+ hashtable->chunk->ntuples += 1;
/* return pointer to the start of the tuple memory */
return ptr;
diff --git a/src/backend/executor/nodeHashjoin.c b/src/backend/executor/nodeHashjoin.c
index b41e4e2..e267bab 100644
--- a/src/backend/executor/nodeHashjoin.c
+++ b/src/backend/executor/nodeHashjoin.c
@@ -21,8 +21,10 @@
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
#include "miscadmin.h"
+#include "pgstat.h"
+#include "storage/barrier.h"
#include "utils/memutils.h"
-
+#include "utils/probes.h"
/*
* States of the ExecHashJoin state machine
@@ -42,11 +44,16 @@
static TupleTableSlot *ExecHashJoinOuterGetTuple(PlanState *outerNode,
HashJoinState *hjstate,
uint32 *hashvalue);
-static TupleTableSlot *ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
- BufFile *file,
+static TupleTableSlot *ExecHashJoinGetSavedTuple(HashJoinTable hashtable,
uint32 *hashvalue,
TupleTableSlot *tupleSlot);
static bool ExecHashJoinNewBatch(HashJoinState *hjstate);
+static void ExecHashJoinLoadBatch(HashJoinState *hjstate);
+static void ExecHashJoinExportAllBatches(HashJoinTable hashtable);
+static void ExecHashJoinExportBatch(HashJoinTable hashtable, int batchno, bool inner);
+static void ExecHashJoinImportBatch(HashJoinTable hashtable,
+ HashJoinBatchReader *reader);
+static void ExecHashJoinPreloadNextBatch(HashJoinState *hjstate);
/* ----------------------------------------------------------------
@@ -147,6 +154,14 @@ ExecHashJoin(HashJoinState *node)
/* no chance to not build the hash table */
node->hj_FirstOuterTupleSlot = NULL;
}
+ else if (hashNode->shared_table_data != NULL)
+ {
+ /*
+ * TODO: The empty-outer optimization is not implemented
+ * for shared hash tables yet.
+ */
+ node->hj_FirstOuterTupleSlot = NULL;
+ }
else if (HJ_FILL_OUTER(node) ||
(outerNode->plan->startup_cost < hashNode->ps.plan->total_cost &&
!node->hj_OuterNotEmpty))
@@ -166,7 +181,7 @@ ExecHashJoin(HashJoinState *node)
/*
* create the hash table
*/
- hashtable = ExecHashTableCreate((Hash *) hashNode->ps.plan,
+ hashtable = ExecHashTableCreate(hashNode,
node->hj_HashOperators,
HJ_FILL_INNER(node));
node->hj_HashTable = hashtable;
@@ -177,12 +192,57 @@ ExecHashJoin(HashJoinState *node)
hashNode->hashtable = hashtable;
(void) MultiExecProcNode((PlanState *) hashNode);
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(&hashtable->shared->barrier) >=
+ PHJ_PHASE_HASHING);
+
+ /*
+ * Check if we are a worker that attached too late to
+ * avoid deadlock risk with the leader, or a leader that
+ * arrived here too late.
+ */
+ if (ExecHashCheckForEarlyExit(hashtable))
+ {
+ /*
+ * Other participants will need to handle all future
+ * batches written by me. We don't detach until after
+ * we've exported all batches, otherwise the phase
+ * might advance and another participant might try to
+ * import them.
+ */
+ if (BarrierPhase(&hashtable->shared->barrier) <=
+ PHJ_PHASE_PROBING)
+ ExecHashJoinExportAllBatches(hashtable);
+ BarrierDetach(&hashtable->shared->barrier);
+ hashtable->detached_early = true;
+ return NULL;
+ }
+
+ /*
+ * Export just the next batch, if there is one, because it
+ * is now read-only and other participants may decide to
+ * read from it. Future batches can still be written to
+ * if work_mem is exceeded by any future batch and we
+ * decide to increase their number, so we can't export
+ * those yet. We'll export the batch files written by
+ * each participant only as they become read-only, but
+ * before any participant reads from them.
+ */
+ if (hashtable->nbatch > 1)
+ {
+ ExecHashJoinExportBatch(hashtable, 1, false);
+ ExecHashJoinExportBatch(hashtable, 1, true);
+ }
+ }
+
/*
* If the inner relation is completely empty, and we're not
* doing a left outer join, we can quit without scanning the
* outer relation.
*/
- if (hashtable->totalTuples == 0 && !HJ_FILL_OUTER(node))
+ if (!HashJoinTableIsShared(hashtable) && /* TODO:TM */
+ hashtable->totalTuples == 0 && !HJ_FILL_OUTER(node))
return NULL;
/*
@@ -198,12 +258,73 @@ ExecHashJoin(HashJoinState *node)
*/
node->hj_OuterNotEmpty = false;
- node->hj_JoinState = HJ_NEED_NEW_OUTER;
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier = &hashtable->shared->barrier;
+ int phase = BarrierPhase(barrier);
+
+ /*
+ * Map the current phase to the appropriate initial state
+ * for this worker, so we can get started.
+ */
+ Assert(BarrierPhase(barrier) >= PHJ_PHASE_PROBING);
+ hashtable->curbatch = PHJ_PHASE_TO_BATCHNO(phase);
+ switch (PHJ_PHASE_TO_SUBPHASE(phase))
+ {
+ case PHJ_SUBPHASE_PROMOTING:
+ /* Wait for serial phase to finish. */
+ BarrierWait(barrier, WAIT_EVENT_HASHJOIN_PROMOTING);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(barrier)) ==
+ PHJ_SUBPHASE_LOADING);
+ /* fall through */
+ case PHJ_SUBPHASE_LOADING:
+ /* Help load the current batch. */
+ ExecHashUpdate(hashtable);
+ ExecHashJoinOpenBatch(hashtable, hashtable->curbatch,
+ true);
+ ExecHashJoinLoadBatch(node);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(barrier)) ==
+ PHJ_SUBPHASE_PROBING);
+ /* fall through */
+ case PHJ_SUBPHASE_PREPARING:
+ /* Wait for serial phase to finish. */
+ BarrierWait(barrier, WAIT_EVENT_HASHJOIN_PROMOTING);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(barrier)) ==
+ PHJ_SUBPHASE_PROBING);
+ /* fall through */
+ case PHJ_SUBPHASE_PROBING:
+ /* Help probe the current batch. */
+ ExecHashUpdate(hashtable);
+ ExecHashJoinOpenBatch(hashtable, hashtable->curbatch,
+ false);
+ node->hj_JoinState = HJ_NEED_NEW_OUTER;
+ break;
+ case PHJ_SUBPHASE_UNMATCHED:
+ /* Help scan for unmatched inner tuples. */
+ ExecHashUpdate(hashtable);
+ node->hj_JoinState = HJ_FILL_INNER_TUPLES;
+ break;
+ }
+ continue;
+ }
+ else
+ {
+ node->hj_JoinState = HJ_NEED_NEW_OUTER;
+ ExecHashJoinOpenBatch(hashtable, 0, false);
+ }
/* FALL THRU */
case HJ_NEED_NEW_OUTER:
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(PHJ_PHASE_TO_BATCHNO(BarrierPhase(&hashtable->shared->barrier)) ==
+ hashtable->curbatch);
+ Assert(PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_PROBING);
+ }
+
/*
* We don't have an outer tuple, try to get the next one
*/
@@ -213,6 +334,67 @@ ExecHashJoin(HashJoinState *node)
if (TupIsNull(outerTupleSlot))
{
/* end of batch, or maybe whole join */
+
+ /*
+ * Switch to reading tuples from the next inner batch. We
+ * do this here because in the shared hash table case we
+ * want to do this before ExecHashJoinPreloadNextBatch.
+ */
+ if (hashtable->curbatch + 1 < hashtable->nbatch)
+ ExecHashJoinOpenBatch(hashtable,
+ hashtable->curbatch + 1,
+ true);
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /*
+ * Check if we are a leader that can't go further than
+ * probing the first batch without deadlock risk,
+ * because there are workers running.
+ */
+ if (ExecHashCheckForEarlyExit(hashtable))
+ {
+ /*
+ * Other backends will need to handle all future
+ * batches written by me. We don't detach until
+ * after we've exported all batches, otherwise
+ * another participant might try to import them
+ * too soon.
+ */
+ ExecHashJoinExportAllBatches(hashtable);
+ BarrierDetach(&hashtable->shared->barrier);
+ hashtable->detached_early = true;
+ return NULL;
+ }
+
+ /*
+ * We may be able to load some amount of the next
+ * batch into spare work_mem, before we start waiting
+ * for other workers to finish probing the current
+ * batch.
+ */
+ ExecHashJoinPreloadNextBatch(node);
+
+ /*
+ * We can't start searching for unmatched tuples until
+ * all participants have finished probing, so we
+ * synchronize here.
+ */
+ if (BarrierWait(&hashtable->shared->barrier,
+ WAIT_EVENT_HASHJOIN_PROBING))
+ {
+ /* Serial phase: prepare for unmatched. */
+ if (HJ_FILL_INNER(node))
+ {
+ hashtable->chunk = NULL;
+ hashtable->shared->chunks_unmatched =
+ hashtable->shared->chunks;
+ hashtable->shared->chunks = InvalidDsaPointer;
+ }
+ }
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_UNMATCHED_BATCH(hashtable->curbatch));
+ }
if (HJ_FILL_INNER(node))
{
/* set up to scan for unmatched inner tuples */
@@ -250,9 +432,9 @@ ExecHashJoin(HashJoinState *node)
* Save it in the corresponding outer-batch file.
*/
Assert(batchno > hashtable->curbatch);
- ExecHashJoinSaveTuple(ExecFetchSlotMinimalTuple(outerTupleSlot),
- hashvalue,
- &hashtable->outerBatchFile[batchno]);
+ ExecHashJoinSaveTuple(hashtable,
+ ExecFetchSlotMinimalTuple(outerTupleSlot),
+ hashvalue, batchno, false);
/* Loop around, staying in HJ_NEED_NEW_OUTER state */
continue;
}
@@ -296,6 +478,13 @@ ExecHashJoin(HashJoinState *node)
if (joinqual == NIL || ExecQual(joinqual, econtext, false))
{
node->hj_MatchedOuter = true;
+ /*
+ * Note: it is OK to do this in a shared hash table
+ * without any kind of memory synchronization, because the
+ * only transition is 0->1, so ordering doesn't matter if
+ * several backends do it, and there will be a memory
+ * barrier before anyone reads it.
+ */
HeapTupleHeaderSetMatch(HJTUPLE_MINTUPLE(node->hj_CurTuple));
/* In an antijoin, we never return a matched tuple */
@@ -631,6 +820,88 @@ ExecEndHashJoin(HashJoinState *node)
ExecEndNode(innerPlanState(node));
}
+void
+ExecShutdownHashJoin(HashJoinState *node)
+{
+ /*
+ * TODO: Figure out how to handle this! For now, just clear the shared
+ * hash table so that ExecEndHashJoin won't blow up when it's called after
+ * the dsa_area has been detached...
+ */
+ if (node->hj_HashTable)
+ node->hj_HashTable->shared = NULL;
+}
+
+/*
+ * For shared hash joins, load as much of the next batch as we can as part of
+ * the probing phase for the current batch. This overlapping means that we do
+ * something useful with a CPU and the spare memory before we start waiting
+ * for other workers.
+ */
+static void
+ExecHashJoinPreloadNextBatch(HashJoinState *hjstate)
+{
+ HashJoinTable hashtable = hjstate->hj_HashTable;
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Barrier *barrier PG_USED_FOR_ASSERTS_ONLY = &hashtable->shared->barrier;
+ int curbatch = hashtable->curbatch;
+ int next_batch = curbatch + 1;
+ TupleTableSlot *slot;
+ uint32 hashvalue;
+
+ Assert(BarrierPhase(barrier) == PHJ_PHASE_PROBING_BATCH(curbatch));
+
+ /*
+ * TODO: We can't preload batch 1 at the end of probing batch 0,
+ * because the leader might call ExecHashJoinExportAllBatches() during
+ * that phase. Batches can't be exported by one backend and imported
+ * and accessed by another in the same phase. Is there a way to
+ * reorder things and avoid that problem?
+ */
+ if (next_batch == 1)
+ return;
+
+ if (next_batch < hashtable->nbatch)
+ {
+ for (;;)
+ {
+ slot = ExecHashJoinGetSavedTuple(hashtable,
+ &hashvalue,
+ hjstate->hj_HashTupleSlot);
+ if (slot == NULL)
+ {
+ /*
+ * We were able to load the whole batch into memory
+ * without running out of work_mem.
+ */
+ break;
+ }
+
+ /*
+ * Try to preload this tuple into a chunk. It is not actually
+ * inserted into the hash table yet.
+ */
+ if (!ExecHashTableInsert(hashtable,
+ hjstate->hj_HashTupleSlot,
+ hashvalue,
+ true)) /* preload */
+ {
+ /*
+ * There is no more work_mem. We'll leave this tuple in
+ * the slot and tell ExecHashJoinLoadBatch to insert it
+ * once we've finish probing the current hash table.
+ */
+ hashtable->preloaded_spare_tuple = true;
+ hashtable->preloaded_spare_tuple_hash = hashvalue;
+ return;
+ }
+ }
+ }
+ }
+}
+
/*
* ExecHashJoinOuterGetTuple
*
@@ -680,7 +951,6 @@ ExecHashJoinOuterGetTuple(PlanState *outerNode,
{
/* remember outer relation is not empty for possible rescan */
hjstate->hj_OuterNotEmpty = true;
-
return slot;
}
@@ -699,11 +969,10 @@ ExecHashJoinOuterGetTuple(PlanState *outerNode,
* In outer-join cases, we could get here even though the batch file
* is empty.
*/
- if (file == NULL)
+ if (!HashJoinTableIsShared(hashtable) && file == NULL)
return NULL;
- slot = ExecHashJoinGetSavedTuple(hjstate,
- file,
+ slot = ExecHashJoinGetSavedTuple(hashtable,
hashvalue,
hjstate->hj_OuterTupleSlot);
if (!TupIsNull(slot))
@@ -726,22 +995,26 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
HashJoinTable hashtable = hjstate->hj_HashTable;
int nbatch;
int curbatch;
- BufFile *innerFile;
- TupleTableSlot *slot;
- uint32 hashvalue;
nbatch = hashtable->nbatch;
curbatch = hashtable->curbatch;
+ if (HashJoinTableIsShared(hashtable))
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_UNMATCHED_BATCH(curbatch));
+
if (curbatch > 0)
{
/*
* We no longer need the previous outer batch file; close it right
* away to free disk space.
*/
+ /* TODO: is this ok for a shared hash table? */
if (hashtable->outerBatchFile[curbatch])
+ {
BufFileClose(hashtable->outerBatchFile[curbatch]);
- hashtable->outerBatchFile[curbatch] = NULL;
+ hashtable->outerBatchFile[curbatch] = NULL;
+ }
}
else /* we just finished the first batch */
{
@@ -776,7 +1049,8 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
* need to be reassigned.
*/
curbatch++;
- while (curbatch < nbatch &&
+ while (!HashJoinTableIsShared(hashtable) &&
+ curbatch < nbatch &&
(hashtable->outerBatchFile[curbatch] == NULL ||
hashtable->innerBatchFile[curbatch] == NULL))
{
@@ -792,13 +1066,15 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
if (hashtable->outerBatchFile[curbatch] &&
nbatch != hashtable->nbatch_outstart)
break; /* must process due to rule 3 */
- /* We can ignore this batch. */
/* Release associated temp files right away. */
+ /* TODO review */
if (hashtable->innerBatchFile[curbatch])
BufFileClose(hashtable->innerBatchFile[curbatch]);
+
hashtable->innerBatchFile[curbatch] = NULL;
if (hashtable->outerBatchFile[curbatch])
BufFileClose(hashtable->outerBatchFile[curbatch]);
+
hashtable->outerBatchFile[curbatch] = NULL;
curbatch++;
}
@@ -812,48 +1088,163 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
* Reload the hash table with the new inner batch (which could be empty)
*/
ExecHashTableReset(hashtable);
+ ExecHashJoinLoadBatch(hjstate);
+
+ return true;
+}
- innerFile = hashtable->innerBatchFile[curbatch];
+static void
+ExecHashJoinLoadBatch(HashJoinState *hjstate)
+{
+ HashJoinTable hashtable = hjstate->hj_HashTable;
+ int curbatch = hashtable->curbatch;
+ TupleTableSlot *slot;
+ uint32 hashvalue;
+
+ TRACE_POSTGRESQL_HASH_LOADING_START();
- if (innerFile != NULL)
+ if (HashJoinTableIsShared(hashtable))
{
- if (BufFileSeek(innerFile, 0, 0L, SEEK_SET))
- ereport(ERROR,
- (errcode_for_file_access(),
- errmsg("could not rewind hash-join temporary file: %m")));
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_LOADING_BATCH(curbatch));
- while ((slot = ExecHashJoinGetSavedTuple(hjstate,
- innerFile,
- &hashvalue,
- hjstate->hj_HashTupleSlot)))
- {
- /*
- * NOTE: some tuples may be sent to future batches. Also, it is
- * possible for hashtable->nbatch to be increased here!
- */
- ExecHashTableInsert(hashtable, slot, hashvalue);
- }
+ /*
+ * Shrinking may be triggered while loading, if work_mem is exceeded.
+ * We need to be attached to shrink_barrier so that we can coordinate
+ * that among participants.
+ */
+ BarrierAttach(&hashtable->shared->shrink_barrier);
+ }
+
+ /*
+ * In HJ_NEED_NEW_OUTER, we already selected the current inner batch for
+ * reading from. If there is a shared hash table, we may have already
+ * partially loaded the hash table in ExecHashJoinPreloadNextBatch. It
+ * may have already loaded one tuple that it couldn't insert, so we'll do
+ * that first.
+ */
+ Assert(hashtable->batch_reader.batchno == curbatch);
+ Assert(hashtable->batch_reader.inner);
+
+ if (hashtable->preloaded_spare_tuple)
+ {
+ bool success;
+
+ Assert(HashJoinTableIsShared(hashtable));
+ Assert(!TupIsNull(hjstate->hj_HashTupleSlot));
+ success = ExecHashTableInsert(hashtable, hjstate->hj_HashTupleSlot,
+ hashtable->preloaded_spare_tuple_hash,
+ false);
+ Assert(success);
+ hashtable->preloaded_spare_tuple = false;
+ }
+
+ /*
+ * If we preloaded any tuples, we now need to insert them into the
+ * hashtable.
+ */
+ ExecHashRebucket(hashtable);
+
+ /* Finally, we can read in the rest of the batch. */
+ for (;;)
+ {
+ slot = ExecHashJoinGetSavedTuple(hashtable,
+ &hashvalue,
+ hjstate->hj_HashTupleSlot);
+
+ if (slot == NULL)
+ break;
/*
- * after we build the hash table, the inner batch file is no longer
- * needed
+ * NOTE: some tuples may be sent to future batches. Also, it is
+ * possible for hashtable->nbatch to be increased here!
*/
- BufFileClose(innerFile);
- hashtable->innerBatchFile[curbatch] = NULL;
+ ExecHashTableInsert(hashtable, slot, hashvalue, false);
+ }
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ /* We have finished any potential shrinking. */
+ BarrierDetach(&hashtable->shared->shrink_barrier);
}
+ TRACE_POSTGRESQL_HASH_LOADING_DONE();
+
/*
- * Rewind outer batch file (if present), so that we can start reading it.
+ * Now that we have finished loading this batch into the hash table, we
+ * can set our outer batch read head to the start of the current batch,
+ * and our inner batch read head to the start of the NEXT batch (as
+ * expected by ExecHashJoinPreloadNextBatch).
*/
- if (hashtable->outerBatchFile[curbatch] != NULL)
+ if (HashJoinTableIsShared(hashtable))
{
- if (BufFileSeek(hashtable->outerBatchFile[curbatch], 0, 0L, SEEK_SET))
- ereport(ERROR,
- (errcode_for_file_access(),
- errmsg("could not rewind hash-join temporary file: %m")));
+ /*
+ * Wait until all participants have finished loading their portion of
+ * the hash table.
+ */
+ if (BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASHJOIN_LOADING))
+ {
+ /* Serial phase: prepare to read this outer and next inner batch */
+ ExecHashJoinRewindBatches(hashtable, hashtable->curbatch);
+ }
+
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_PREPARING_BATCH(hashtable->curbatch));
+ /*
+ * Since we have finished loading the current batch into memory, the
+ * batch files generated by this participant for the next batch are
+ * now read-only. So it's time to export them for other participants
+ * to read from if they run out of tuples to read from their own batch
+ * files. We'll export the current outer batch, so that it can be
+ * used for probing, and the next inner batch so that it can be used
+ * for preloading tuples for the next batch when that is finished.
+ */
+ ExecHashJoinExportBatch(hashtable, hashtable->curbatch, false);
+ if (hashtable->curbatch + 1 < hashtable->nbatch)
+ ExecHashJoinExportBatch(hashtable, hashtable->curbatch + 1, true);
+
+ BarrierWait(&hashtable->shared->barrier, WAIT_EVENT_HASHJOIN_PREPARING);
+ Assert(BarrierPhase(&hashtable->shared->barrier) ==
+ PHJ_PHASE_PROBING_BATCH(hashtable->curbatch));
}
+ else
+ ExecHashJoinRewindBatches(hashtable, hashtable->curbatch);
- return true;
+ /*
+ * The inner batch file is no longer needed by any participant, because
+ * the hash table has been fully reloaded.
+ */
+ ExecHashJoinCloseBatch(hashtable, hashtable->curbatch, true);
+
+ /* Prepare to read from the current outer batch. */
+ ExecHashJoinOpenBatch(hashtable, hashtable->curbatch, false);
+}
+
+/*
+ * Export a BufFile, copy the descriptor to DSA memory and return the
+ * dsa_pointer.
+ */
+static dsa_pointer
+make_batch_descriptor(dsa_area *area, BufFile *file)
+{
+ dsa_pointer pointer;
+ BufFileDescriptor *source;
+ BufFileDescriptor *target;
+ size_t size;
+
+ source = BufFileExport(file);
+ size = BufFileDescriptorSize(source);
+ pointer = dsa_allocate(area, size);
+ if (!DsaPointerIsValid(pointer))
+ ereport(ERROR,
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of memory"),
+ errdetail("Failed on dsa_allocate of size %zu.", size)));
+ target = dsa_get_address(area, pointer);
+ memcpy(target, source, size);
+ pfree(source);
+
+ return pointer;
}
/*
@@ -868,17 +1259,26 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
* will get messed up.
*/
void
-ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
- BufFile **fileptr)
+ExecHashJoinSaveTuple(HashJoinTable hashtable,
+ MinimalTuple tuple, uint32 hashvalue,
+ int batchno,
+ bool inner)
{
- BufFile *file = *fileptr;
+ BufFile *file;
size_t written;
+ if (inner)
+ file = hashtable->innerBatchFile[batchno];
+ else
+ file = hashtable->outerBatchFile[batchno];
if (file == NULL)
{
/* First write to this batch file, so open it. */
file = BufFileCreateTemp(false);
- *fileptr = file;
+ if (inner)
+ hashtable->innerBatchFile[batchno] = file;
+ else
+ hashtable->outerBatchFile[batchno] = file;
}
written = BufFileWrite(file, (void *) &hashvalue, sizeof(uint32));
@@ -892,57 +1292,519 @@ ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
ereport(ERROR,
(errcode_for_file_access(),
errmsg("could not write to hash-join temporary file: %m")));
+
+ TRACE_POSTGRESQL_HASH_SAVE_TUPLE(HashJoinParticipantNumber(),
+ batchno,
+ inner);
+}
+
+/*
+ * Export the inner or outer batch file written by this participant for a
+ * given batch number, so that other backends can import and read from it if
+ * they run out of tuples to read from their own files. This must be done
+ * after this participant has finished writing to the batch, but before any
+ * other participant might attempt to read from it.
+ */
+static void
+ExecHashJoinExportBatch(HashJoinTable hashtable, int batchno, bool inner)
+{
+ HashJoinParticipantState *participant;
+ BufFile *file;
+
+ TRACE_POSTGRESQL_HASHJOIN_EXPORT_BATCH(HashJoinParticipantNumber(),
+ batchno,
+ inner);
+
+ Assert(HashJoinTableIsShared(hashtable));
+ Assert(batchno < hashtable->nbatch);
+
+ participant = &hashtable->shared->participants[HashJoinParticipantNumber()];
+
+ /* We will export batches one-by-one. */
+ participant->nbatch = -1;
+
+ if (inner)
+ {
+ participant->inner_batchno = batchno;
+ file = hashtable->innerBatchFile[batchno];
+ if (file != NULL)
+ participant->inner_batch_descriptor =
+ make_batch_descriptor(hashtable->area, file);
+ else
+ participant->inner_batch_descriptor =
+ InvalidDsaPointer;
+ }
+ else
+ {
+ participant->outer_batchno = batchno;
+ file = hashtable->outerBatchFile[batchno];
+ if (file != NULL)
+ participant->outer_batch_descriptor =
+ make_batch_descriptor(hashtable->area, file);
+ else
+ participant->outer_batch_descriptor =
+ InvalidDsaPointer;
+ }
+}
+
+/*
+ * Export all future batches. This must be called by any backend that exits
+ * early, to make sure that the batch files it wrote to can be consumed by
+ * other participants.
+ */
+static void
+ExecHashJoinExportAllBatches(HashJoinTable hashtable)
+{
+ HashJoinParticipantState *participant;
+ dsa_pointer *inner_batch_descriptors;
+ dsa_pointer *outer_batch_descriptors;
+ Size size;
+ BufFile *file;
+ int i;
+
+ /*
+ * Sanity check that we are in one of the expected phases, in which no
+ * other participant could be reading the state we are writing.
+ *
+ * TODO: See ExecHashJoinPreloadNextBatch where we can't actually preload
+ * batch 1 because of this. Need to figure something better out.
+ *
+ */
+ Assert(BarrierPhase(&hashtable->shared->barrier) == PHJ_PHASE_HASHING ||
+ BarrierPhase(&hashtable->shared->barrier) == PHJ_PHASE_PROBING);
+
+ TRACE_POSTGRESQL_HASHJOIN_EXPORT_ALL_BATCHES(HashJoinParticipantNumber(),
+ hashtable->nbatch);
+
+ /* If we didn't generate any batches there is nothing to do. */
+ participant = &hashtable->shared->participants[HashJoinParticipantNumber()];
+ if (hashtable->nbatch <= 1)
+ {
+ /* No one ever needs to read batch 0. */
+ participant->nbatch = 0;
+ return;
+ }
+
+ /* Set up space for descriptors for all my batches. */
+ participant->nbatch = hashtable->nbatch;
+ size = sizeof(dsa_pointer) * hashtable->nbatch;
+ participant->inner_batch_descriptors = dsa_allocate(hashtable->area, size);
+ participant->outer_batch_descriptors = dsa_allocate(hashtable->area, size);
+ if (!DsaPointerIsValid(participant->inner_batch_descriptors) ||
+ !DsaPointerIsValid(participant->outer_batch_descriptors))
+ ereport(ERROR,
+ (errcode(ERRCODE_OUT_OF_MEMORY),
+ errmsg("out of memory"),
+ errdetail("Failed on dsa_allocate of size %zu.", size)));
+ inner_batch_descriptors =
+ dsa_get_address(hashtable->area,
+ participant->inner_batch_descriptors);
+ outer_batch_descriptors =
+ dsa_get_address(hashtable->area,
+ participant->outer_batch_descriptors);
+ memset(inner_batch_descriptors, 0, size);
+ memset(outer_batch_descriptors, 0, size);
+
+ /* Now export all batches that were written by this participant. */
+ for (i = hashtable->curbatch + 1; i < hashtable->nbatch; ++i)
+ {
+ file = hashtable->innerBatchFile[i];
+ if (file != NULL)
+ inner_batch_descriptors[i] =
+ make_batch_descriptor(hashtable->area, file);
+ file = hashtable->outerBatchFile[i];
+ if (file != NULL)
+ outer_batch_descriptors[i] =
+ make_batch_descriptor(hashtable->area, file);
+ }
+}
+
+/*
+ * Import a batch that was exported by another participant, so that this
+ * process can read it. The participant and batch numbers should be already
+ * set in the reader object that is passed in.
+ */
+static void
+ExecHashJoinImportBatch(HashJoinTable hashtable, HashJoinBatchReader *reader)
+{
+ dsa_pointer descriptor = InvalidDsaPointer;
+ HashJoinParticipantState *participant;
+
+ TRACE_POSTGRESQL_HASHJOIN_IMPORT_BATCH(reader->participant_number,
+ reader->batchno,
+ reader->inner);
+
+ Assert(reader->participant_number >= 0 &&
+ reader->participant_number < hashtable->shared->planned_participants);
+
+ /* Find the participant referenced by the reader. */
+ participant = &hashtable->shared->participants[reader->participant_number];
+
+ /* Find the descriptor exported by that participant for that batch. */
+ if (participant->nbatch != -1)
+ {
+ /* It exported all its batches and left. Find the correct one. */
+ if (reader->batchno < participant->nbatch)
+ {
+ dsa_pointer *descriptors;
+
+ Assert(DsaPointerIsValid(participant->inner_batch_descriptors));
+ Assert(DsaPointerIsValid(participant->outer_batch_descriptors));
+ descriptors =
+ dsa_get_address(hashtable->area,
+ reader->inner
+ ? participant->inner_batch_descriptors
+ : participant->outer_batch_descriptors);
+ if (DsaPointerIsValid(descriptors[reader->batchno]))
+ descriptor = descriptors[reader->batchno];
+ }
+ }
+ else
+ {
+ /* It must have just exported the exact batch we expect. */
+ Assert((reader->inner &&
+ (reader->batchno == participant->inner_batchno)) ||
+ (!reader->inner &&
+ (reader->batchno == participant->outer_batchno)));
+
+ if (reader->inner)
+ descriptor = participant->inner_batch_descriptor;
+ else
+ descriptor = participant->outer_batch_descriptor;
+ }
+
+ /* Import the BufFile, if we found one. */
+ if (DsaPointerIsValid(descriptor))
+ {
+ reader->head.fileno = reader->head.offset = -1;
+ reader->file = BufFileImport(dsa_get_address(hashtable->area,
+ descriptor));
+ if (reader->inner)
+ reader->shared = &participant->inner_batch_reader;
+ else
+ reader->shared = &participant->outer_batch_reader;
+ Assert(reader->shared->batchno == reader->batchno);
+ }
+ else
+ {
+ reader->file = NULL;
+ reader->shared = NULL;
+ }
+}
+
+/*
+ * Select the batch file that ExecHashJoinGetSavedTuple will read from.
+ */
+void
+ExecHashJoinOpenBatch(HashJoinTable hashtable, int batchno, bool inner)
+{
+ HashJoinBatchReader *batch_reader = &hashtable->batch_reader;
+
+ TRACE_POSTGRESQL_HASHJOIN_OPEN_BATCH(HashJoinParticipantNumber(),
+ batchno,
+ inner);
+
+ if (batchno == 0)
+ batch_reader->file = NULL;
+ else
+ batch_reader->file = inner
+ ? hashtable->innerBatchFile[batchno]
+ : hashtable->outerBatchFile[batchno];
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ HashJoinParticipantState *participant;
+
+ /* Initially we will read from the caller's batch file. */
+ participant =
+ &hashtable->shared->participants[HashJoinParticipantNumber()];
+ batch_reader->shared = inner
+ ? &participant->inner_batch_reader
+ : &participant->outer_batch_reader;
+ /* Seek to the shared position at next read. */
+ batch_reader->head.fileno = -1;
+ batch_reader->head.offset = -1;
+ }
+ else
+ {
+ batch_reader->shared = NULL;
+ /* Seek to start of batch now, if there is one. */
+ if (batch_reader->file != NULL)
+ BufFileSeek(batch_reader->file, 0, 0, SEEK_SET);
+ }
+
+ batch_reader->participant_number = HashJoinParticipantNumber();
+ batch_reader->batchno = batchno;
+ batch_reader->inner = inner;
+}
+
+/*
+ * Close a batch, once it is not needed by any participant. This causes batch
+ * files created by this participant to be deleted.
+ */
+void
+ExecHashJoinCloseBatch(HashJoinTable hashtable, int batchno, bool inner)
+{
+ HashJoinParticipantState *participant;
+ HashJoinBatchReader *batch_reader;
+ BufFile *file;
+
+ /*
+ * We only need to close the batch owned by THIS participant. That causes
+ * it to be deleted. Batches opened in this backend but created by other
+ * participants are closed by ExecHashJoinGetSavedTuple when it reaches
+ * the end of the file, allowing them to be closed sooner.
+ */
+ batch_reader = &hashtable->batch_reader;
+ participant = &hashtable->shared->participants[HashJoinParticipantNumber()];
+ if (inner)
+ {
+ file = hashtable->innerBatchFile[batchno];
+ hashtable->innerBatchFile[batchno] = NULL;
+ }
+ else
+ {
+ file = hashtable->outerBatchFile[batchno];
+ hashtable->outerBatchFile[batchno] = NULL;
+ }
+ if (file == NULL)
+ return;
+
+ Assert(batch_reader->file == NULL || file == batch_reader->file);
+
+ BufFileClose(file);
+ batch_reader->file = NULL;
+}
+
+/*
+ * Rewind batch readers. The outer batch reader is rewound to the start of
+ * batchno. The inner batch reader is rewound to the start of batchno + 1, in
+ * anticipation of preloading the next batch.
+ */
+void
+ExecHashJoinRewindBatches(HashJoinTable hashtable, int batchno)
+{
+ HashJoinBatchReader *batch_reader;
+ int i;
+
+ batch_reader = &hashtable->batch_reader;
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert(BarrierPhase(&hashtable->shared->barrier) == PHJ_PHASE_CREATING ||
+ (PHJ_PHASE_TO_SUBPHASE(BarrierPhase(&hashtable->shared->barrier)) ==
+ PHJ_SUBPHASE_PREPARING &&
+ PHJ_PHASE_TO_BATCHNO(BarrierPhase(&hashtable->shared->barrier)) ==
+ batchno));
+
+ /* Position the shared read heads for each participant's batch. */
+ for (i = 0; i < hashtable->shared->planned_participants; ++i)
+ {
+ HashJoinSharedBatchReader *reader;
+
+ reader = &hashtable->shared->participants[i].outer_batch_reader;
+ reader->batchno = batchno; /* for probing this batch */
+ reader->head.fileno = 0;
+ reader->head.offset = 0;
+
+ reader = &hashtable->shared->participants[i].inner_batch_reader;
+ reader->batchno = batchno + 1; /* for preloading the next batch */
+ reader->head.fileno = 0;
+ reader->head.offset = 0;
+ }
+ }
}
/*
* ExecHashJoinGetSavedTuple
- * read the next tuple from a batch file. Return NULL if no more.
+ * read the next tuple from the batch selected with
+ * ExecHashJoinOpenBatch, including the batch files of
+ * other participants if the hash table is shared. Return NULL if no
+ * more.
*
* On success, *hashvalue is set to the tuple's hash value, and the tuple
* itself is stored in the given slot.
*/
static TupleTableSlot *
-ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
- BufFile *file,
+ExecHashJoinGetSavedTuple(HashJoinTable hashtable,
uint32 *hashvalue,
TupleTableSlot *tupleSlot)
{
- uint32 header[2];
- size_t nread;
- MinimalTuple tuple;
+ TupleTableSlot *result = NULL;
+ HashJoinBatchReader *batch_reader = &hashtable->batch_reader;
- /*
- * Since both the hash value and the MinimalTuple length word are uint32,
- * we can read them both in one BufFileRead() call without any type
- * cheating.
- */
- nread = BufFileRead(file, (void *) header, sizeof(header));
- if (nread == 0) /* end of file */
+ for (;;)
{
- ExecClearTuple(tupleSlot);
- return NULL;
- }
- if (nread != sizeof(header))
- ereport(ERROR,
- (errcode_for_file_access(),
- errmsg("could not read from hash-join temporary file: %m")));
- *hashvalue = header[0];
- tuple = (MinimalTuple) palloc(header[1]);
- tuple->t_len = header[1];
- nread = BufFileRead(file,
- (void *) ((char *) tuple + sizeof(uint32)),
- header[1] - sizeof(uint32));
- if (nread != header[1] - sizeof(uint32))
- ereport(ERROR,
- (errcode_for_file_access(),
+ uint32 header[2];
+ size_t nread;
+ MinimalTuple tuple;
+ bool can_close = false;
+
+ if (batch_reader->file == NULL)
+ {
+ /*
+ * No file found for the current participant. Try stealing tuples
+ * from the next participant.
+ */
+ goto next_participant;
+ }
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ Assert((batch_reader->inner &&
+ batch_reader->shared ==
+ &hashtable->shared->participants[batch_reader->participant_number].inner_batch_reader) ||
+ (!batch_reader->inner &&
+ batch_reader->shared ==
+ &hashtable->shared->participants[batch_reader->participant_number].outer_batch_reader));
+
+ LWLockAcquire(&batch_reader->shared->lock, LW_EXCLUSIVE);
+ Assert(batch_reader->shared->batchno == batch_reader->batchno);
+ if (batch_reader->shared->error)
+ {
+ /* Don't try to read if reading failed in some other backend. */
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from hash-join temporary file")));
+ }
+
+ /* Set the shared error flag, which we'll clear if we succeed. */
+ batch_reader->shared->error = true;
+
+ /*
+ * If another worker has moved the shared read head since we last read,
+ * we'll need to seek to the new shared position.
+ */
+ if (batch_reader->head.fileno != batch_reader->shared->head.fileno ||
+ batch_reader->head.offset != batch_reader->shared->head.offset)
+ {
+ TRACE_POSTGRESQL_HASH_SEEK(HashJoinParticipantNumber(),
+ batch_reader->participant_number,
+ batch_reader->batchno,
+ batch_reader->inner,
+ batch_reader->shared->head.fileno,
+ batch_reader->shared->head.offset);
+ BufFileSeek(batch_reader->file,
+ batch_reader->shared->head.fileno,
+ batch_reader->shared->head.offset,
+ SEEK_SET);
+ batch_reader->head = batch_reader->shared->head;
+ }
+ }
+
+ /* Try to read the size and hash. */
+ nread = BufFileRead(batch_reader->file, (void *) header, sizeof(header));
+ if (nread > 0)
+ {
+ if (nread != sizeof(header))
+ {
+ ereport(ERROR,
+ (errcode_for_file_access(),
errmsg("could not read from hash-join temporary file: %m")));
- return ExecStoreMinimalTuple(tuple, tupleSlot, true);
-}
+ }
+ *hashvalue = header[0];
+ tuple = (MinimalTuple) palloc(header[1]);
+ tuple->t_len = header[1];
+ nread = BufFileRead(batch_reader->file,
+ (void *) ((char *) tuple + sizeof(uint32)),
+ header[1] - sizeof(uint32));
+ if (nread != header[1] - sizeof(uint32))
+ {
+ ereport(ERROR,
+ (errcode_for_file_access(),
+ errmsg("could not read from hash-join temporary file: %m")));
+ }
+
+ TRACE_POSTGRESQL_HASH_GET_SAVED_TUPLE(HashJoinParticipantNumber(),
+ batch_reader->participant_number,
+ batch_reader->batchno,
+ batch_reader->inner);
+ result = ExecStoreMinimalTuple(tuple, tupleSlot, true);
+ }
+
+ if (HashJoinTableIsShared(hashtable))
+ {
+ if (nread == 0 &&
+ batch_reader->participant_number !=
+ HashJoinParticipantNumber())
+ {
+ /*
+ * We've reached the end of another paticipant's batch file,
+ * so close it now. We'll deal with closing THIS
+ * participant's batch file later, because we don't want the
+ * files to be deleted just yet.
+ */
+ can_close = true;
+ }
+ /* Commit new head position to shared memory and clear error. */
+ BufFileTell(batch_reader->file,
+ &batch_reader->head.fileno,
+ &batch_reader->head.offset);
+ batch_reader->shared->head = batch_reader->head;
+ batch_reader->shared->error = false;
+ if (nread == 0)
+ TRACE_POSTGRESQL_HASH_TELL(HashJoinParticipantNumber(),
+ batch_reader->participant_number,
+ batch_reader->batchno,
+ batch_reader->inner,
+ batch_reader->shared->head.fileno,
+ batch_reader->shared->head.offset);
+ LWLockRelease(&batch_reader->shared->lock);
+ }
+
+ if (can_close)
+ {
+ BufFileClose(batch_reader->file);
+ batch_reader->file = NULL;
+ }
+
+ if (result != NULL)
+ return result;
+
+next_participant:
+ if (!HashJoinTableIsShared(hashtable))
+ {
+ /* Private hash table, end of batch. */
+ ExecClearTuple(tupleSlot); /* TODO:TM also needed for shared n'est-ce pas? */
+ return NULL;
+ }
+
+ /* Try the next participant's batch file. */
+ batch_reader->participant_number =
+ (batch_reader->participant_number + 1) %
+ hashtable->shared->planned_participants;
+ if (batch_reader->participant_number == HashJoinParticipantNumber())
+ {
+ /*
+ * We've made it all the way back to the file we started with,
+ * which is the one that this backend wrote. So there are no more
+ * tuples to be had in any participant's batch file.
+ */
+ ExecClearTuple(tupleSlot);
+ return NULL;
+ }
+ /* Import the BufFile from that participant, if it exported one. */
+ ExecHashJoinImportBatch(hashtable, batch_reader);
+ }
+}
void
ExecReScanHashJoin(HashJoinState *node)
{
+ HashState *hashNode = (HashState *) innerPlanState(node);
+
+ /* We can't use HashJoinTableIsShared if the table is NULL. */
+ if (hashNode->shared_table_data != NULL)
+ {
+ elog(ERROR, "TODO: shared ExecReScanHashJoin not yet implemented");
+
+ /* Coordinate a rewind to the shared hash table creation phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier,
+ PHJ_PHASE_BEGINNING,
+ WAIT_EVENT_HASHJOIN_REWINDING);
+ }
+
/*
* In a multi-batch join, we currently have to do rescans the hard way,
* primarily because batch temp files may have already been released. But
@@ -977,6 +1839,14 @@ ExecReScanHashJoin(HashJoinState *node)
/* ExecHashJoin can skip the BUILD_HASHTABLE step */
node->hj_JoinState = HJ_NEED_NEW_OUTER;
+
+ if (HashJoinTableIsShared(node->hj_HashTable))
+ {
+ /* Coordinate a rewind to the shared probing phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier,
+ PHJ_PHASE_PROBING,
+ WAIT_EVENT_HASHJOIN_REWINDING2);
+ }
}
else
{
@@ -985,6 +1855,14 @@ ExecReScanHashJoin(HashJoinState *node)
node->hj_HashTable = NULL;
node->hj_JoinState = HJ_BUILD_HASHTABLE;
+ if (HashJoinTableIsShared(node->hj_HashTable))
+ {
+ /* Coordinate a rewind to the shared hash table creation phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier,
+ PHJ_PHASE_BEGINNING,
+ WAIT_EVENT_HASHJOIN_REWINDING3);
+ }
+
/*
* if chgParam of subnode is not null then plan will be re-scanned
* by first ExecProcNode.
@@ -1011,3 +1889,97 @@ ExecReScanHashJoin(HashJoinState *node)
if (node->js.ps.lefttree->chgParam == NULL)
ExecReScan(node->js.ps.lefttree);
}
+
+void ExecHashJoinEstimate(HashJoinState *state, ParallelContext *pcxt)
+{
+ size_t size;
+
+ size = offsetof(SharedHashJoinTableData, participants) +
+ sizeof(HashJoinParticipantState) * (pcxt->nworkers + 1);
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+void
+ExecHashJoinInitializeDSM(HashJoinState *state, ParallelContext *pcxt)
+{
+ HashState *hashNode;
+ SharedHashJoinTable shared;
+ size_t size;
+ int planned_participants;
+ int i;
+
+ /*
+ * Disable shared hash table mode if we failed to create a real DSM
+ * segment, because that means that we don't have a DSA area to work
+ * with.
+ */
+ if (pcxt->seg == NULL)
+ return;
+
+ /*
+ * Set up the state needed to coordinate access to the shared hash table,
+ * using the plan node ID as the toc key.
+ */
+ planned_participants = pcxt->nworkers + 1; /* possible workers + leader */
+ size = offsetof(SharedHashJoinTableData, participants) +
+ sizeof(HashJoinParticipantState) * planned_participants;
+ shared = shm_toc_allocate(pcxt->toc, size);
+ BarrierInit(&shared->barrier, 0);
+ BarrierInit(&shared->shrink_barrier, 0);
+ shared->buckets = InvalidDsaPointer;
+ shared->chunks = InvalidDsaPointer;
+ shared->chunks_preloaded = InvalidDsaPointer;
+ shared->chunks_to_rebucket = InvalidDsaPointer;
+ shared->chunks_to_shrink = InvalidDsaPointer;
+ shared->chunks_unmatched = InvalidDsaPointer;
+ shared->planned_participants = planned_participants;
+ shared->size = 0;
+ shared->size_preloaded = 0;
+ shared->shrinking_enabled = true;
+ shm_toc_insert(pcxt->toc, state->js.ps.plan->plan_node_id, shared);
+
+ /* Initialize the LWLocks. */
+ LWLockInitialize(&shared->chunk_lock, LWTRANCHE_PARALLEL_HASH_JOIN_CHUNK);
+ for (i = 0; i < planned_participants; ++i)
+ {
+ LWLockInitialize(&shared->participants[i].inner_batch_reader.lock,
+ LWTRANCHE_PARALLEL_HASH_JOIN_INNER_BATCH_READER);
+ LWLockInitialize(&shared->participants[i].outer_batch_reader.lock,
+ LWTRANCHE_PARALLEL_HASH_JOIN_OUTER_BATCH_READER);
+ }
+
+ /*
+ * Pass the SharedHashJoinTable to the hash node. If the Gather node
+ * running in the leader backend decides to execute the hash join, it
+ * hasn't called ExecHashJoinInitializeWorker so it doesn't have
+ * state->shared_table_data set up. So we must do it here.
+ */
+ hashNode = (HashState *) innerPlanState(state);
+ hashNode->shared_table_data = shared;
+}
+
+void
+ExecHashJoinInitializeWorker(HashJoinState *state, shm_toc *toc)
+{
+ HashState *hashNode;
+
+ state->hj_sharedHashJoinTable =
+ shm_toc_lookup(toc, state->js.ps.plan->plan_node_id);
+
+ /*
+ * Inject SharedHashJoinTable into the hash node. It could instead have
+ * its own ExecHashInitializeWorker function, but we only want to set its
+ * 'parallel_aware' flag if we want to tell it to actually build the hash
+ * table in parallel. Since its parallel_aware flag also controls whether
+ * its 'InitializeWorker' function gets called, and it also needs access
+ * to this object for serial shared hash mode, we'll pass it on here
+ * instead of depending on that.
+ */
+ hashNode = (HashState *) innerPlanState(state);
+ hashNode->shared_table_data = state->hj_sharedHashJoinTable;
+ Assert(hashNode->shared_table_data != NULL);
+
+ Assert(HashJoinParticipantNumber() <
+ hashNode->shared_table_data->planned_participants);
+}
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 439a946..df1d574 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -31,6 +31,8 @@
#include "executor/nodeSeqscan.h"
#include "utils/rel.h"
+#include <unistd.h>
+
static void InitScanRelation(SeqScanState *node, EState *estate, int eflags);
static TupleTableSlot *SeqNext(SeqScanState *node);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 806d0a9..a2beb27 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1993,6 +1993,7 @@ _outHashPath(StringInfo str, const HashPath *node)
WRITE_NODE_FIELD(path_hashclauses);
WRITE_INT_FIELD(num_batches);
+ WRITE_ENUM_FIELD(table_type, HashPathTableType);
}
static void
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a52eb7e..2856bcd 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -104,6 +104,7 @@
double seq_page_cost = DEFAULT_SEQ_PAGE_COST;
double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
+double cpu_shared_tuple_cost = DEFAULT_CPU_SHARED_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
double parallel_tuple_cost = DEFAULT_PARALLEL_TUPLE_COST;
@@ -2693,16 +2694,19 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
List *hashclauses,
Path *outer_path, Path *inner_path,
SpecialJoinInfo *sjinfo,
- SemiAntiJoinFactors *semifactors)
+ SemiAntiJoinFactors *semifactors,
+ HashPathTableType table_type)
{
Cost startup_cost = 0;
Cost run_cost = 0;
double outer_path_rows = outer_path->rows;
double inner_path_rows = inner_path->rows;
+ double inner_path_rows_total = inner_path_rows;
int num_hashclauses = list_length(hashclauses);
int numbuckets;
int numbatches;
int num_skew_mcvs;
+ size_t space_allowed; /* not used */
/* cost of source data */
startup_cost += outer_path->startup_cost;
@@ -2724,8 +2728,43 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
run_cost += cpu_operator_cost * num_hashclauses * outer_path_rows;
/*
+ * If this is a shared hash table, there is a extra charge for inserting
+ * each tuple into the shared hash table, to cover the overhead of memory
+ * synchronization that makes the hash table slightly slower to build than
+ * a private hash table. There is no extra charge for probing the hash
+ * table for outer path row, on the basis that read-only access to the
+ * hash table shouldn't generate any extra memory synchronization.
+ *
+ * cpu_shared_tuple_cost acts a tie-breaker controlling whether we prefer
+ * HASH_TABLE_PRIVATE or HASH_TABLE_SHARED_SERIAL plans, when the hash
+ * table fits in work_mem, since the cost is otherwise the same. If it is
+ * positive, then we'll prefer private hash tables, even though that means
+ * that we'll be running N copies of the inner plan. Running N copies of
+ * the copies of the inner plan in parallel is not considered more
+ * expensive than running 1 copy of the inner plan while N-1 participants
+ * do nothing, despite doing less work in total.
+ */
+ if (table_type != HASHPATH_TABLE_PRIVATE)
+ startup_cost += cpu_shared_tuple_cost * inner_path_rows;
+
+ /*
+ * If this is a parallel shared hash table, then the value we have for
+ * inner_rows 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 (table_type == HASHPATH_TABLE_SHARED_PARALLEL)
+ inner_path_rows_total *= outer_path->parallel_workers + 1;
+
+ /*
* Get hash table size that executor would use for inner relation.
*
+ * Shared hash tables are allowed to be larger to make up for the fact
+ * that there is only one copy shared by all parallel query participants,
+ * which may reduce the number of batches. That means that
+ * HASH_TABLE_SHARED_SERIAL is likely to beat HASH_TABLE_PRIVATE when we
+ * expect to exceed work_mem.
+ *
* XXX for the moment, always assume that skew optimization will be
* performed. As long as SKEW_WORK_MEM_PERCENT is small, it's not worth
* trying to determine that for sure.
@@ -2733,9 +2772,12 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
* XXX at some point it might be interesting to try to account for skew
* optimization in the cost estimate, but for now, we don't.
*/
- ExecChooseHashTableSize(inner_path_rows,
+ ExecChooseHashTableSize(inner_path_rows_total,
inner_path->pathtarget->width,
true, /* useskew */
+ table_type != HASHPATH_TABLE_PRIVATE, /* shared */
+ outer_path->parallel_workers,
+ &space_allowed,
&numbuckets,
&numbatches,
&num_skew_mcvs);
@@ -2746,12 +2788,19 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
* time. Charge seq_page_cost per page, since the I/O should be nice and
* sequential. Writing the inner rel counts as startup cost, all the rest
* as run cost.
+ *
+ * If the hash table is HASH_TABLE_PRIVATE, then every participant will
+ * write a copy of every batch file, but this happens in parallel so we
+ * don't consider that to be more expensive than the
+ * HASH_TABLE_SHARED_SERIAL case where only one participant does that. It
+ * is not clear how the costing should be affected by higher disk
+ * bandwidth usage.
*/
if (numbatches > 1)
{
double outerpages = page_size(outer_path_rows,
outer_path->pathtarget->width);
- double innerpages = page_size(inner_path_rows,
+ double innerpages = page_size(inner_path_rows_total,
inner_path->pathtarget->width);
startup_cost += seq_page_cost * innerpages;
diff --git a/src/backend/optimizer/path/joinpath.c b/src/backend/optimizer/path/joinpath.c
index 7c30ec6..209b9d1 100644
--- a/src/backend/optimizer/path/joinpath.c
+++ b/src/backend/optimizer/path/joinpath.c
@@ -492,7 +492,8 @@ try_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *hashclauses,
JoinType jointype,
- JoinPathExtraData *extra)
+ JoinPathExtraData *extra,
+ HashPathTableType table_type)
{
Relids required_outer;
JoinCostWorkspace workspace;
@@ -517,7 +518,7 @@ try_hashjoin_path(PlannerInfo *root,
*/
initial_cost_hashjoin(root, &workspace, jointype, hashclauses,
outer_path, inner_path,
- extra->sjinfo, &extra->semifactors);
+ extra->sjinfo, &extra->semifactors, table_type);
if (add_path_precheck(joinrel,
workspace.startup_cost, workspace.total_cost,
@@ -534,7 +535,8 @@ try_hashjoin_path(PlannerInfo *root,
inner_path,
extra->restrictlist,
required_outer,
- hashclauses));
+ hashclauses,
+ table_type));
}
else
{
@@ -555,7 +557,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *hashclauses,
JoinType jointype,
- JoinPathExtraData *extra)
+ JoinPathExtraData *extra,
+ HashPathTableType table_type)
{
JoinCostWorkspace workspace;
@@ -580,7 +583,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
*/
initial_cost_hashjoin(root, &workspace, jointype, hashclauses,
outer_path, inner_path,
- extra->sjinfo, &extra->semifactors);
+ extra->sjinfo, &extra->semifactors,
+ table_type);
if (!add_partial_path_precheck(joinrel, workspace.total_cost, NIL))
return;
@@ -596,7 +600,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
inner_path,
extra->restrictlist,
NULL,
- hashclauses));
+ hashclauses,
+ table_type));
}
/*
@@ -1401,7 +1406,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
/* no possibility of cheap startup here */
}
else if (jointype == JOIN_UNIQUE_INNER)
@@ -1417,7 +1423,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
if (cheapest_startup_outer != NULL &&
cheapest_startup_outer != cheapest_total_outer)
try_hashjoin_path(root,
@@ -1426,7 +1433,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
}
else
{
@@ -1447,7 +1455,8 @@ hash_inner_and_outer(PlannerInfo *root,
cheapest_total_inner,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
foreach(lc1, outerrel->cheapest_parameterized_paths)
{
@@ -1481,7 +1490,8 @@ hash_inner_and_outer(PlannerInfo *root,
innerpath,
hashclauses,
jointype,
- extra);
+ extra,
+ HASHPATH_TABLE_PRIVATE);
}
}
}
@@ -1490,23 +1500,32 @@ hash_inner_and_outer(PlannerInfo *root,
* If the joinrel is parallel-safe, we may be able to consider a
* partial hash join. However, we can't handle JOIN_UNIQUE_OUTER,
* because the outer path will be partial, and therefore we won't be
- * able to properly guarantee uniqueness. Similarly, we can't handle
- * JOIN_FULL and JOIN_RIGHT, because they can produce false null
- * extended rows. Also, the resulting path must not be parameterized.
+ * able to properly guarantee uniqueness. Also, the resulting path
+ * must not be parameterized.
*/
if (joinrel->consider_parallel &&
save_jointype != JOIN_UNIQUE_OUTER &&
- save_jointype != JOIN_FULL &&
- save_jointype != JOIN_RIGHT &&
outerrel->partial_pathlist != NIL &&
bms_is_empty(joinrel->lateral_relids))
{
Path *cheapest_partial_outer;
+ 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? */
+ if (innerrel->partial_pathlist != NIL)
+ cheapest_partial_inner =
+ (Path *) linitial(innerrel->partial_pathlist);
+ if (cheapest_partial_inner != NULL)
+ try_partial_hashjoin_path(root, joinrel,
+ cheapest_partial_outer,
+ cheapest_partial_inner,
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_SHARED_PARALLEL);
+
/*
* Normally, given that the joinrel is parallel-safe, the cheapest
* total inner path will also be parallel-safe, but if not, we'll
@@ -1534,10 +1553,27 @@ hash_inner_and_outer(PlannerInfo *root,
}
if (cheapest_safe_inner != NULL)
+ {
+ /* Try a shared table with only one worker building the table. */
try_partial_hashjoin_path(root, joinrel,
cheapest_partial_outer,
cheapest_safe_inner,
- hashclauses, jointype, extra);
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_SHARED_SERIAL);
+ /*
+ * Also try private hash tables, built by each worker, but
+ * only if it's not a FULL or RIGHT join. Those rely on being
+ * able to track which hash table entries have been matched,
+ * but we don't have a way to unify the HEAP_TUPLE_HAS_MATCH
+ * flags from all the private copies of the hash table.
+ */
+ if (save_jointype != JOIN_FULL && save_jointype != JOIN_RIGHT)
+ try_partial_hashjoin_path(root, joinrel,
+ cheapest_partial_outer,
+ cheapest_safe_inner,
+ hashclauses, jointype, extra,
+ HASHPATH_TABLE_PRIVATE);
+ }
}
}
}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c7bcd9b..cac4932 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -3938,6 +3938,23 @@ create_hashjoin_plan(PlannerInfo *root,
copy_plan_costsize(&hash_plan->plan, inner_plan);
hash_plan->plan.startup_cost = hash_plan->plan.total_cost;
+ /*
+ * Set the table as sharable if appropriate, with parallel or serial
+ * building.
+ */
+ switch (best_path->table_type)
+ {
+ case HASHPATH_TABLE_SHARED_PARALLEL:
+ hash_plan->shared_table = true;
+ hash_plan->plan.parallel_aware = true;
+ break;
+ case HASHPATH_TABLE_SHARED_SERIAL:
+ hash_plan->shared_table = true;
+ break;
+ case HASHPATH_TABLE_PRIVATE:
+ break;
+ }
+
join_plan = make_hashjoin(tlist,
joinclauses,
otherclauses,
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3b7c56d..a1d7b20 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -2096,6 +2096,7 @@ create_mergejoin_path(PlannerInfo *root,
* 'required_outer' is the set of required outer rels
* 'hashclauses' are the RestrictInfo nodes to use as hash clauses
* (this should be a subset of the restrict_clauses list)
+ * 'table_type' for level of hash table sharing
*/
HashPath *
create_hashjoin_path(PlannerInfo *root,
@@ -2108,7 +2109,8 @@ create_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *restrict_clauses,
Relids required_outer,
- List *hashclauses)
+ List *hashclauses,
+ HashPathTableType table_type)
{
HashPath *pathnode = makeNode(HashPath);
@@ -2123,9 +2125,13 @@ create_hashjoin_path(PlannerInfo *root,
sjinfo,
required_outer,
&restrict_clauses);
- pathnode->jpath.path.parallel_aware = false;
+ pathnode->jpath.path.parallel_aware =
+ joinrel->consider_parallel &&
+ (table_type == HASHPATH_TABLE_SHARED_SERIAL ||
+ table_type == HASHPATH_TABLE_SHARED_PARALLEL);
pathnode->jpath.path.parallel_safe = joinrel->consider_parallel &&
outer_path->parallel_safe && inner_path->parallel_safe;
+ pathnode->table_type = table_type;
/* This is a foolish way to estimate parallel_workers, but for now... */
pathnode->jpath.path.parallel_workers = outer_path->parallel_workers;
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index f37a0bf..d562fef 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -3392,6 +3392,63 @@ pgstat_get_wait_ipc(WaitEventIPC w)
case WAIT_EVENT_SYNC_REP:
event_name = "SyncRep";
break;
+ case WAIT_EVENT_HASH_CREATING:
+ event_name = "Hash/Creating";
+ break;
+ case WAIT_EVENT_HASH_HASHING:
+ event_name = "Hash/Hashing";
+ break;
+ case WAIT_EVENT_HASH_SHRINKING1:
+ event_name = "Hash/Shrinking1";
+ break;
+ case WAIT_EVENT_HASH_SHRINKING2:
+ event_name = "Hash/Shrinking2";
+ break;
+ case WAIT_EVENT_HASH_SHRINKING3:
+ event_name = "Hash/Shrinking3";
+ break;
+ case WAIT_EVENT_HASH_SHRINKING4:
+ event_name = "Hash/Shrinking4";
+ break;
+ case WAIT_EVENT_HASH_RESIZING:
+ event_name = "Hash/Resizing";
+ break;
+ case WAIT_EVENT_HASH_REBUCKETING:
+ event_name = "Hash/Rebucketing";
+ break;
+ case WAIT_EVENT_HASH_BEGINNING:
+ event_name = "Hash/Beginning";
+ break;
+ case WAIT_EVENT_HASH_DESTROY:
+ event_name = "Hash/Destroy";
+ break;
+ case WAIT_EVENT_HASH_UNMATCHED:
+ event_name = "Hash/Unmatched";
+ break;
+ case WAIT_EVENT_HASH_PROMOTING:
+ event_name = "Hash/Promoting";
+ break;
+ case WAIT_EVENT_HASHJOIN_PROMOTING:
+ event_name = "HashJoin/Promoting";
+ break;
+ case WAIT_EVENT_HASHJOIN_PREPARING:
+ event_name = "HashJoin/Preparing";
+ break;
+ case WAIT_EVENT_HASHJOIN_PROBING:
+ event_name = "HashJoin/Probing";
+ break;
+ case WAIT_EVENT_HASHJOIN_LOADING:
+ event_name = "HashJoin/Loading";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING:
+ event_name = "HashJoin/Rewinding";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING2:
+ event_name = "HashJoin/Rewinding2";;
+ break;
+ case WAIT_EVENT_HASHJOIN_REWINDING3:
+ event_name = "HashJoin/Rewinding3";;
+ break;
/* no default case, so that compiler will warn */
}
diff --git a/src/backend/storage/file/buffile.c b/src/backend/storage/file/buffile.c
index 7ebd636..18ffd4e 100644
--- a/src/backend/storage/file/buffile.c
+++ b/src/backend/storage/file/buffile.c
@@ -40,8 +40,11 @@
#include "storage/fd.h"
#include "storage/buffile.h"
#include "storage/buf_internals.h"
+#include "utils/probes.h"
#include "utils/resowner.h"
+extern int ParallelWorkerNumber;
+
/*
* We break BufFiles into gigabyte-sized segments, regardless of RELSEG_SIZE.
* The reason is that we'd like large temporary BufFiles to be spread across
@@ -89,6 +92,24 @@ struct BufFile
char buffer[BLCKSZ];
};
+/*
+ * Serialized representation of a single file managed by a BufFile.
+ */
+typedef struct BufFileFileDescriptor
+{
+ char path[MAXPGPATH];
+} BufFileFileDescriptor;
+
+/*
+ * Serialized representation of a BufFile, to be created by BufFileExport and
+ * consumed by BufFileImport.
+ */
+struct BufFileDescriptor
+{
+ size_t num_files;
+ BufFileFileDescriptor files[FLEXIBLE_ARRAY_MEMBER];
+};
+
static BufFile *makeBufFile(File firstfile);
static void extendBufFile(BufFile *file);
static void BufFileLoadBuffer(BufFile *file);
@@ -178,6 +199,81 @@ BufFileCreateTemp(bool interXact)
return file;
}
+/*
+ * Export a BufFile description in a serialized form so that another backend
+ * can attach to it and read from it. The format is opaque, but it may be
+ * bitwise copied, and its size may be obtained with BufFileDescriptorSize().
+ */
+BufFileDescriptor *
+BufFileExport(BufFile *file)
+{
+ BufFileDescriptor *descriptor;
+ int i;
+
+ /* Flush output from local buffers. */
+ BufFileFlush(file);
+
+ /* Create and fill in a descriptor. */
+ descriptor = palloc0(offsetof(BufFileDescriptor, files) +
+ sizeof(BufFileFileDescriptor) * file->numFiles);
+ descriptor->num_files = file->numFiles;
+ for (i = 0; i < descriptor->num_files; ++i)
+ {
+ TRACE_POSTGRESQL_BUFFILE_EXPORT_FILE(FilePathName(file->files[i]));
+ strcpy(descriptor->files[i].path, FilePathName(file->files[i]));
+ }
+
+ return descriptor;
+}
+
+/*
+ * Return the size in bytes of a BufFileDescriptor, so that it can be copied.
+ */
+size_t
+BufFileDescriptorSize(const BufFileDescriptor *descriptor)
+{
+ return offsetof(BufFileDescriptor, files) +
+ sizeof(BufFileFileDescriptor) * descriptor->num_files;
+}
+
+/*
+ * Open a BufFile that was created by another backend and then exported. The
+ * file must be read-only in all backends, and is still owned by the backend
+ * that created it. This provides a way for cooperating backends to share
+ * immutable temporary data such as hash join batches.
+ */
+BufFile *
+BufFileImport(BufFileDescriptor *descriptor)
+{
+ BufFile *file = (BufFile *) palloc0(sizeof(BufFile));
+ int i;
+
+ file->numFiles = descriptor->num_files;
+ file->files = (File *) palloc0(sizeof(File) * descriptor->num_files);
+ file->offsets = (off_t *) palloc0(sizeof(off_t) * descriptor->num_files);
+ file->isTemp = false;
+ file->isInterXact = true; /* prevent cleanup by this backend */
+ file->dirty = false;
+ file->resowner = CurrentResourceOwner;
+ file->curFile = 0;
+ file->curOffset = 0L;
+ file->pos = 0;
+ file->nbytes = 0;
+
+ for (i = 0; i < descriptor->num_files; ++i)
+ {
+ TRACE_POSTGRESQL_BUFFILE_IMPORT_FILE(descriptor->files[i].path);
+ file->files[i] =
+ PathNameOpenFile(descriptor->files[i].path,
+ O_RDONLY | PG_BINARY, 0600);
+ if (file->files[i] <= 0)
+ elog(ERROR, "failed to import \"%s\": %m",
+ descriptor->files[i].path);
+ }
+
+ return file;
+}
+
#ifdef NOT_USED
/*
* Create a BufFile and attach it to an already-opened virtual File.
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 1cf0684..833b059 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -510,6 +510,12 @@ RegisterLWLockTranches(void)
"predicate_lock_manager");
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_HASH_JOIN_INNER_BATCH_READER,
+ "hash_join_inner_batches");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_HASH_JOIN_OUTER_BATCH_READER,
+ "hash_join_outer_batches");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_HASH_JOIN_CHUNK,
+ "hash_join_chunk");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 5b23dbf..fdb6d24 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2855,6 +2855,16 @@ static struct config_real ConfigureNamesReal[] =
NULL, NULL, NULL
},
{
+ {"cpu_shared_tuple_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "sharing each tuple with other parallel workers."),
+ NULL
+ },
+ &cpu_shared_tuple_cost,
+ DEFAULT_CPU_SHARED_TUPLE_COST, -DBL_MAX, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
{"cpu_index_tuple_cost", PGC_USERSET, QUERY_TUNING_COST,
gettext_noop("Sets the planner's estimate of the cost of "
"processing each index entry during an index scan."),
diff --git a/src/backend/utils/probes.d b/src/backend/utils/probes.d
index 146fce9..3239c3c 100644
--- a/src/backend/utils/probes.d
+++ b/src/backend/utils/probes.d
@@ -60,6 +60,40 @@ provider postgresql {
probe sort__start(int, bool, int, int, bool);
probe sort__done(bool, long);
+ probe hash__leader__early__exit();
+ probe hash__worker__early__exit();
+ probe hash__hashing__start();
+ probe hash__hashing__done();
+ probe hash__loading__start();
+ probe hash__loading__done();
+ probe hash__increase__buckets(int);
+ probe hash__increase__batches(int);
+ probe hash__shrink__start(int);
+ probe hash__shrink__done();
+ probe hash__shrink__chunk();
+ probe hash__shrink__disabled();
+ probe hash__shrink__stats(size_t, size_t, size_t, size_t);
+ probe hash__rebucket__start();
+ probe hash__rebucket__done(int);
+ probe hash__free__chunk(size_t);
+ probe hash__allocate__chunk(size_t);
+ probe hash__save__tuple(int, int, int);
+ probe hash__get__saved__tuple(int, int, int, int);
+ probe hash__seek(int, int, int, int, int, size_t);
+ probe hash__tell(int, int, int, int, int, size_t);
+ probe hash__insert(int);
+ probe hash__probe(int, int);
+
+ probe hashjoin__start();
+ probe hashjoin__done();
+ probe hashjoin__export__all__batches(int, int);
+ probe hashjoin__export__batch(int, int, bool);
+ probe hashjoin__import__batch(int, int, bool);
+ probe hashjoin__open__batch(int, int, bool);
+
+ probe buffile__import__file(const char *);
+ probe buffile__export__file(const char *);
+
probe buffer__read__start(ForkNumber, BlockNumber, Oid, Oid, Oid, int, bool);
probe buffer__read__done(ForkNumber, BlockNumber, Oid, Oid, Oid, int, bool, bool);
probe buffer__flush__start(ForkNumber, BlockNumber, Oid, Oid, Oid);
diff --git a/src/include/executor/hashjoin.h b/src/include/executor/hashjoin.h
index ac84053..2effc77 100644
--- a/src/include/executor/hashjoin.h
+++ b/src/include/executor/hashjoin.h
@@ -15,7 +15,13 @@
#define HASHJOIN_H
#include "nodes/execnodes.h"
+#include "port/atomics.h"
+#include "storage/barrier.h"
#include "storage/buffile.h"
+#include "storage/fd.h"
+#include "storage/lwlock.h"
+#include "storage/spin.h"
+#include "utils/dsa.h"
/* ----------------------------------------------------------------
* hash-join hash table structures
@@ -63,7 +69,12 @@
typedef struct HashJoinTupleData
{
- struct HashJoinTupleData *next; /* link to next tuple in same bucket */
+ /* link to next tuple in same bucket */
+ union
+ {
+ dsa_pointer shared;
+ struct HashJoinTupleData *private;
+ } next;
uint32 hashvalue; /* tuple's hash code */
/* Tuple data, in MinimalTuple format, follows on a MAXALIGN boundary */
} HashJoinTupleData;
@@ -94,7 +105,12 @@ typedef struct HashJoinTupleData
typedef struct HashSkewBucket
{
uint32 hashvalue; /* common hash value */
- HashJoinTuple tuples; /* linked list of inner-relation tuples */
+ /* linked list of inner-relation tuples */
+ union
+ {
+ dsa_pointer shared;
+ HashJoinTuple private;
+ } tuples;
} HashSkewBucket;
#define SKEW_BUCKET_OVERHEAD MAXALIGN(sizeof(HashSkewBucket))
@@ -103,8 +119,9 @@ typedef struct HashSkewBucket
#define SKEW_MIN_OUTER_FRACTION 0.01
/*
- * To reduce palloc overhead, the HashJoinTuples for the current batch are
- * packed in 32kB buffers instead of pallocing each tuple individually.
+ * To reduce palloc/dsa_allocate overhead, the HashJoinTuples for the current
+ * batch are packed in 32kB buffers instead of pallocing each tuple
+ * individually.
*/
typedef struct HashMemoryChunkData
{
@@ -112,17 +129,137 @@ 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
+ {
+ dsa_pointer shared;
+ struct HashMemoryChunkData *private;
+ } 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_THRESHOLD (HASH_CHUNK_SIZE / 4)
+/*
+ * Read head position in a shared batch file.
+ */
+typedef struct HashJoinBatchPosition
+{
+ int fileno;
+ off_t offset;
+} HashJoinBatchPosition;
+
+/*
+ * The state exposed in shared memory by each participant to coordinate
+ * reading of batch files that it wrote.
+ */
+typedef struct HashJoinSharedBatchReader
+{
+ int batchno; /* the batch number we are currently reading */
+
+ LWLock lock; /* protects access to the members below */
+ bool error; /* has an IO error occurred? */
+ HashJoinBatchPosition head; /* shared read head for current batch */
+} HashJoinSharedBatchReader;
+
+/*
+ * The state exposed in shared memory by each participant allowing its batch
+ * files to be read by other participants.
+ */
+typedef struct HashJoinParticipantState
+{
+ /*
+ * To allow other participants to read from this participant's batch
+ * files, this participant publishes its batch descriptors (or invalid
+ * pointers) here.
+ */
+ int inner_batchno;
+ int outer_batchno;
+ dsa_pointer inner_batch_descriptor;
+ dsa_pointer outer_batch_descriptor;
+
+ /*
+ * In the case of participants that exit early, they must publish all
+ * their future batches, rather than publishing them one by one above.
+ * These point to an array of dsa_pointers to BufFileDescriptor objects.
+ */
+ int nbatch;
+ dsa_pointer inner_batch_descriptors;
+ dsa_pointer outer_batch_descriptors;
+
+ /*
+ * The shared state used to coordinate reading from the current batch. We
+ * need separate objects for the outer and inner side, because in the
+ * probing phase some participants can be reading from the outer batch,
+ * while others can be reading from the inner side to preload the next
+ * batch.
+ */
+ HashJoinSharedBatchReader inner_batch_reader;
+ HashJoinSharedBatchReader outer_batch_reader;
+} HashJoinParticipantState;
+
+/*
+ * The state used by each backend to manage reading from batch files written
+ * by all participants.
+ */
+typedef struct HashJoinBatchReader
+{
+ int participant_number; /* read which participant's batch? */
+ int batchno; /* which batch are we reading? */
+ bool inner; /* inner or outer? */
+ HashJoinSharedBatchReader *shared; /* holder of the shared read head */
+ BufFile *file; /* the file opened in this backend */
+ HashJoinBatchPosition head; /* local read head position */
+} HashJoinBatchReader;
+
+/*
+ * State for a shared hash join table. Each backend participating in a hash
+ * join with a shared hash table also has a HashJoinTableData object in
+ * backend-private memory, which points to this shared state in the DSM
+ * segment.
+ */
+typedef struct SharedHashJoinTableData
+{
+ Barrier barrier; /* synchronization for the whole join */
+ Barrier shrink_barrier; /* synchronization to shrink hashtable */
+ dsa_pointer buckets; /* primary hash table */
+ bool at_least_one_worker; /* did at least one worker join in time? */
+ int nbuckets;
+ int nbuckets_optimal;
+ int nbatch;
+
+ LWLock chunk_lock; /* protects the following members */
+ dsa_pointer chunks; /* chunks loaded for the current batch */
+ dsa_pointer chunks_preloaded; /* chunks preloaded for the next batch */
+ dsa_pointer chunks_to_rebucket; /* chunks with tuples to insert */
+ dsa_pointer chunks_to_shrink; /* chunks needing to be thinned out */
+ dsa_pointer chunks_unmatched; /* chunks for unmatched scanning */
+ Size tuples_this_batch; /* number of tuples in chunks */
+ Size tuples_next_batch; /* number of tuples in chunks_preloaded */
+ Size tuples_in_memory; /* shared counter while rebatching */
+ Size tuples_written_out; /* shared counter while rebatching */
+ Size size; /* size of buckets + chunks */
+ Size size_preloaded; /* size of chunks_preloaded */
+ bool shrinking_enabled;
+
+ int planned_participants; /* number of planned workers + leader */
+
+ /* state exposed by each participant for sharing batches */
+ HashJoinParticipantState participants[FLEXIBLE_ARRAY_MEMBER];
+} SharedHashJoinTableData;
+
+typedef union HashJoinBucketHead
+{
+ dsa_pointer_atomic shared;
+ HashJoinTuple private;
+} HashJoinBucketHead;
+
typedef struct HashJoinTableData
{
int nbuckets; /* # buckets in the in-memory hash table */
@@ -134,7 +271,7 @@ typedef struct HashJoinTableData
int log2_nbuckets_optimal; /* log2(nbuckets_optimal) */
/* buckets[i] is head of list of tuples in i'th in-memory bucket */
- struct HashJoinTupleData **buckets;
+ HashJoinBucketHead *buckets;
/* buckets array is per-batch storage, as are all the tuples */
bool keepNulls; /* true to store unmatchable NULL tuples */
@@ -185,7 +322,84 @@ typedef struct HashJoinTableData
MemoryContext batchCxt; /* context for this-batch-only storage */
/* used for dense allocation of tuples (into linked chunks) */
- HashMemoryChunk chunks; /* one list for the whole batch */
+ HashMemoryChunk chunk; /* current chunk */
+ HashMemoryChunk chunk_preload; /* current chunk for next batch */
+ HashMemoryChunk chunks_to_rebucket; /* after resizing table */
+ HashMemoryChunk chunks_to_shrink;
+ int chunk_unmatched_pos; /* head when scanning for unmatched tuples */
+
+ /* State for coordinating shared tables for parallel hash joins. */
+ dsa_area *area;
+ SharedHashJoinTableData *shared; /* the shared state */
+ int attached_at_phase; /* the phase this participant joined */
+ bool detached_early; /* did we decide to detach early? */
+ HashJoinBatchReader batch_reader; /* state for reading batches in */
+ bool preloaded_spare_tuple; /* is there an extra preloaded tuple? */
+ uint32 preloaded_spare_tuple_hash; /* the tuple's hash value if so */
+ dsa_pointer chunk_shared; /* DSA pointer to 'chunk' */
+ dsa_pointer chunk_preload_shared; /* DSA pointer to 'chunk_preload' */
+
} HashJoinTableData;
+/* Check if a HashJoinTable is shared by parallel workers. */
+#define HashJoinTableIsShared(table) ((table)->shared != NULL)
+
+/* The phases of a parallel hash join. */
+#define PHJ_PHASE_BEGINNING 0
+#define PHJ_PHASE_CREATING 1
+#define PHJ_PHASE_HASHING 2
+#define PHJ_PHASE_RESIZING 3
+#define PHJ_PHASE_REBUCKETING 4
+#define PHJ_PHASE_PROBING 5 /* PHJ_PHASE_PROBING_BATCH(0) */
+#define PHJ_PHASE_UNMATCHED 6 /* PHJ_PHASE_UNMATCHED_BATCH(0) */
+
+/* The subphases for batches. */
+#define PHJ_SUBPHASE_PROMOTING 0
+#define PHJ_SUBPHASE_LOADING 1
+#define PHJ_SUBPHASE_PREPARING 2
+#define PHJ_SUBPHASE_PROBING 3
+#define PHJ_SUBPHASE_UNMATCHED 4
+
+/* The phases of parallel processing for batch(n). */
+#define PHJ_PHASE_PROMOTING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 4)
+#define PHJ_PHASE_LOADING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 3)
+#define PHJ_PHASE_PREPARING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 2)
+#define PHJ_PHASE_PROBING_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 1)
+#define PHJ_PHASE_UNMATCHED_BATCH(n) (PHJ_PHASE_UNMATCHED + (n) * 5 - 0)
+
+/* Phase number -> sub-phase within a batch. */
+#define PHJ_PHASE_TO_SUBPHASE(p) \
+ (((int)(p) - PHJ_PHASE_UNMATCHED + PHJ_SUBPHASE_UNMATCHED) % 5)
+
+/* Phase number -> batch number. */
+#define PHJ_PHASE_TO_BATCHNO(p) \
+ (((int)(p) - PHJ_PHASE_UNMATCHED + PHJ_SUBPHASE_UNMATCHED) / 5)
+
+/*
+ * Is a given phase one in which a new hash table array is being assigned by
+ * one elected backend? That includes initial creation, reallocation during
+ * resize, and promotion of secondary hash table to primary. Workers that
+ * show up and attach at an arbitrary time must wait such phases out before
+ * doing anything with the hash table.
+ */
+#define PHJ_PHASE_MUTATING_TABLE(p) \
+ ((p) == PHJ_PHASE_CREATING || \
+ (p) == PHJ_PHASE_RESIZING || \
+ (PHJ_PHASE_TO_BATCHNO(p) > 0 && \
+ PHJ_PHASE_TO_SUBPHASE(p) == PHJ_SUBPHASE_PROMOTING))
+
+/* The phases of ExecHashShrink. */
+#define PHJ_SHRINK_PHASE_BEGINNING 0
+#define PHJ_SHRINK_PHASE_CLEARING 1
+#define PHJ_SHRINK_PHASE_WORKING 2
+#define PHJ_SHRINK_PHASE_DECIDING 3
+
+/*
+ * Return the 'participant number' for a process participating in a parallel
+ * hash join. We give a number < hashtable->shared->planned_participants
+ * to each potential participant, including the leader.
+ */
+#define HashJoinParticipantNumber() \
+ (IsParallelWorker() ? ParallelWorkerNumber + 1 : 0)
+
#endif /* HASHJOIN_H */
diff --git a/src/include/executor/nodeHash.h b/src/include/executor/nodeHash.h
index fe5c264..a7a5c6e 100644
--- a/src/include/executor/nodeHash.h
+++ b/src/include/executor/nodeHash.h
@@ -22,12 +22,12 @@ extern Node *MultiExecHash(HashState *node);
extern void ExecEndHash(HashState *node);
extern void ExecReScanHash(HashState *node);
-extern HashJoinTable ExecHashTableCreate(Hash *node, List *hashOperators,
+extern HashJoinTable ExecHashTableCreate(HashState *node, List *hashOperators,
bool keepNulls);
extern void ExecHashTableDestroy(HashJoinTable hashtable);
-extern void ExecHashTableInsert(HashJoinTable hashtable,
+extern bool ExecHashTableInsert(HashJoinTable hashtable,
TupleTableSlot *slot,
- uint32 hashvalue);
+ uint32 hashvalue, bool secondary);
extern bool ExecHashGetHashValue(HashJoinTable hashtable,
ExprContext *econtext,
List *hashkeys,
@@ -45,9 +45,14 @@ extern bool ExecScanHashTableForUnmatched(HashJoinState *hjstate,
extern void ExecHashTableReset(HashJoinTable hashtable);
extern void ExecHashTableResetMatchFlags(HashJoinTable hashtable);
extern void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
+ bool shared, int parallel_workers,
+ size_t *spaceAllowed,
int *numbuckets,
int *numbatches,
int *num_skew_mcvs);
extern int ExecHashGetSkewBucket(HashJoinTable hashtable, uint32 hashvalue);
+extern void ExecHashUpdate(HashJoinTable hashtable);
+extern bool ExecHashCheckForEarlyExit(HashJoinTable hashtable);
+extern void ExecHashRebucket(HashJoinTable hashtable);
#endif /* NODEHASH_H */
diff --git a/src/include/executor/nodeHashjoin.h b/src/include/executor/nodeHashjoin.h
index ddc32b1..ef7d935 100644
--- a/src/include/executor/nodeHashjoin.h
+++ b/src/include/executor/nodeHashjoin.h
@@ -14,15 +14,28 @@
#ifndef NODEHASHJOIN_H
#define NODEHASHJOIN_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
#include "storage/buffile.h"
+#include "storage/shm_toc.h"
extern HashJoinState *ExecInitHashJoin(HashJoin *node, EState *estate, int eflags);
extern TupleTableSlot *ExecHashJoin(HashJoinState *node);
extern void ExecEndHashJoin(HashJoinState *node);
+extern void ExecShutdownHashJoin(HashJoinState *node);
extern void ExecReScanHashJoin(HashJoinState *node);
-extern void ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
- BufFile **fileptr);
+extern void ExecHashJoinSaveTuple(HashJoinTable hashtable,
+ MinimalTuple tuple, uint32 hashvalue,
+ int batchno, bool inner);
+extern void ExecHashJoinRewindBatches(HashJoinTable hashtable, int batchno);
+extern void ExecHashJoinOpenBatch(HashJoinTable hashtable,
+ int batchno, bool inner);
+extern void ExecHashJoinCloseBatch(HashJoinTable hashtable,
+ int batchno, bool inner);
+
+extern void ExecHashJoinEstimate(HashJoinState *state, ParallelContext *pcxt);
+extern void ExecHashJoinInitializeDSM(HashJoinState *state, ParallelContext *pcxt);
+extern void ExecHashJoinInitializeWorker(HashJoinState *state, shm_toc *toc);
#endif /* NODEHASHJOIN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ce13bf7..deb8497 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "utils/dsa.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1755,6 +1756,7 @@ typedef struct MergeJoinState
/* these structs are defined in executor/hashjoin.h: */
typedef struct HashJoinTupleData *HashJoinTuple;
typedef struct HashJoinTableData *HashJoinTable;
+typedef struct SharedHashJoinTableData *SharedHashJoinTable;
typedef struct HashJoinState
{
@@ -1776,6 +1778,7 @@ typedef struct HashJoinState
int hj_JoinState;
bool hj_MatchedOuter;
bool hj_OuterNotEmpty;
+ SharedHashJoinTable hj_sharedHashJoinTable;
} HashJoinState;
@@ -2006,6 +2009,9 @@ typedef struct HashState
HashJoinTable hashtable; /* hash table for the hashjoin */
List *hashkeys; /* list of ExprState nodes */
/* hashkeys is same as parent's hj_InnerHashKeys */
+
+ /* The following are the same as the parent's. */
+ SharedHashJoinTable shared_table_data;
} HashState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 692a626..6d1460b 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -782,6 +782,7 @@ typedef struct Hash
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 */
+ bool shared_table; /* table shared by multiple workers? */
/* all other info is in the parent HashJoin node */
} Hash;
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index e1d31c7..43f9515 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1258,6 +1258,16 @@ typedef struct MergePath
bool materialize_inner; /* add Materialize to inner? */
} MergePath;
+typedef enum
+{
+ /* Every worker builds its own private copy of the hash table. */
+ HASHPATH_TABLE_PRIVATE,
+ /* One worker builds a shared hash table, and all workers probe it. */
+ HASHPATH_TABLE_SHARED_SERIAL,
+ /* All workers build a shared hash table, and then probe it. */
+ HASHPATH_TABLE_SHARED_PARALLEL
+} HashPathTableType;
+
/*
* A hashjoin path has these fields.
*
@@ -1272,6 +1282,7 @@ typedef struct HashPath
JoinPath jpath;
List *path_hashclauses; /* join clauses used for hashing */
int num_batches; /* number of batches expected */
+ HashPathTableType table_type; /* level of sharedness */
} HashPath;
/*
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..220c013 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -24,6 +24,7 @@
#define DEFAULT_SEQ_PAGE_COST 1.0
#define DEFAULT_RANDOM_PAGE_COST 4.0
#define DEFAULT_CPU_TUPLE_COST 0.01
+#define DEFAULT_CPU_SHARED_TUPLE_COST 0.001
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
#define DEFAULT_PARALLEL_TUPLE_COST 0.1
@@ -48,6 +49,7 @@ typedef enum
extern PGDLLIMPORT double seq_page_cost;
extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
+extern PGDLLIMPORT double cpu_shared_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
extern PGDLLIMPORT double parallel_tuple_cost;
@@ -144,7 +146,8 @@ extern void initial_cost_hashjoin(PlannerInfo *root,
List *hashclauses,
Path *outer_path, Path *inner_path,
SpecialJoinInfo *sjinfo,
- SemiAntiJoinFactors *semifactors);
+ SemiAntiJoinFactors *semifactors,
+ HashPathTableType table_type);
extern void final_cost_hashjoin(PlannerInfo *root, HashPath *path,
JoinCostWorkspace *workspace,
SpecialJoinInfo *sjinfo,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index d16f879..42633c5 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -134,7 +134,8 @@ extern HashPath *create_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *restrict_clauses,
Relids required_outer,
- List *hashclauses);
+ List *hashclauses,
+ HashPathTableType table_type);
extern ProjectionPath *create_projection_path(PlannerInfo *root,
RelOptInfo *rel,
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 5b37894..f54b0a5 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -785,7 +785,26 @@ typedef enum
WAIT_EVENT_MQ_SEND,
WAIT_EVENT_PARALLEL_FINISH,
WAIT_EVENT_SAFE_SNAPSHOT,
- WAIT_EVENT_SYNC_REP
+ WAIT_EVENT_SYNC_REP,
+ WAIT_EVENT_HASH_CREATING,
+ WAIT_EVENT_HASH_HASHING,
+ WAIT_EVENT_HASH_RESIZING,
+ WAIT_EVENT_HASH_REBUCKETING,
+ WAIT_EVENT_HASH_BEGINNING,
+ WAIT_EVENT_HASH_DESTROY,
+ WAIT_EVENT_HASH_UNMATCHED,
+ WAIT_EVENT_HASH_PROMOTING,
+ WAIT_EVENT_HASH_SHRINKING1,
+ WAIT_EVENT_HASH_SHRINKING2,
+ WAIT_EVENT_HASH_SHRINKING3,
+ WAIT_EVENT_HASH_SHRINKING4,
+ WAIT_EVENT_HASHJOIN_PROMOTING,
+ WAIT_EVENT_HASHJOIN_PROBING,
+ WAIT_EVENT_HASHJOIN_LOADING,
+ WAIT_EVENT_HASHJOIN_PREPARING,
+ WAIT_EVENT_HASHJOIN_REWINDING,
+ WAIT_EVENT_HASHJOIN_REWINDING2, /* TODO: rename me */
+ WAIT_EVENT_HASHJOIN_REWINDING3 /* TODO: rename me */
} WaitEventIPC;
/* ----------
diff --git a/src/include/storage/buffile.h b/src/include/storage/buffile.h
index fe00bf0..023eb3f 100644
--- a/src/include/storage/buffile.h
+++ b/src/include/storage/buffile.h
@@ -30,12 +30,17 @@
typedef struct BufFile BufFile;
+typedef struct BufFileDescriptor BufFileDescriptor;
+
/*
* prototypes for functions in buffile.c
*/
extern BufFile *BufFileCreateTemp(bool interXact);
extern void BufFileClose(BufFile *file);
+extern BufFileDescriptor *BufFileExport(BufFile *file);
+extern BufFile *BufFileImport(BufFileDescriptor *descriptor);
+extern size_t BufFileDescriptorSize(const BufFileDescriptor *descriptor);
extern size_t BufFileRead(BufFile *file, void *ptr, size_t size);
extern size_t BufFileWrite(BufFile *file, void *ptr, size_t size);
extern int BufFileSeek(BufFile *file, int fileno, off_t offset, int whence);
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 8bd93c3..dd6d48e 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -211,6 +211,9 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_BUFFER_MAPPING,
LWTRANCHE_LOCK_MANAGER,
LWTRANCHE_PREDICATE_LOCK_MANAGER,
+ LWTRANCHE_PARALLEL_HASH_JOIN_INNER_BATCH_READER,
+ LWTRANCHE_PARALLEL_HASH_JOIN_OUTER_BATCH_READER,
+ LWTRANCHE_PARALLEL_HASH_JOIN_CHUNK,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
On Sat, Jan 7, 2017 at 9:01 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
On Tue, Jan 3, 2017 at 10:53 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:I will post a new rebased version soon with that and
some other nearby problems fixed.Here is a new WIP patch.
I forgot to mention: this applies on top of barrier-v5.patch, over here:
/messages/by-id/CAEepm=3g3EC734kgriWseiJPfUQZeoMWdhAfzOc0ecewAa5uXg@mail.gmail.com
--
Thomas Munro
http://www.enterprisedb.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sat, Jan 7, 2017 at 9:01 AM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
On Tue, Jan 3, 2017 at 10:53 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:I will post a new rebased version soon with that and
some other nearby problems fixed.Here is a new WIP patch.
To make this easier to understand and harmonise the logic used in a
few places, I'm now planning to chop it up into a patch series,
probably something like this:
1. Change existing hash join code to use chunk-based accounting
2. Change existing hash join code to use a new interface for dealing
with batches
3. Add shared hash join support, single batch only
4. Add components for doing shared batch reading (unused)
5. Add multi-batch shared hash join support
--
Thomas Munro
http://www.enterprisedb.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Jan 6, 2017 at 12:01 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
Here is a new WIP patch. I have plenty of things to tidy up (see note
at end), but the main ideas are now pretty clear and I'd appreciate
some feedback.
I have some review feedback for your V3. I've chosen to start with the
buffile.c stuff, since of course it might share something with my
parallel tuplesort patch. This isn't comprehensive, but I will have
more comprehensive feedback soon.
I'm not surprised that you've generally chosen to make shared BufFile
management as simple as possible, with no special infrastructure other
than the ability to hold open other backend temp files concurrently
within a worker, and no writing to another worker's temp file, or
shared read pointer. As you put it, everything is immutable. I
couldn't see much opportunity for adding a lot of infrastructure that
wasn't written explicitly as parallel hash join code/infrastructure.
My sense is that that was a good decision. I doubted that you'd ever
want some advanced, generic shared BufFile thing with multiple read
pointers, built-in cache coherency, etc. (Robert seemed to think that
you'd go that way, though.)
Anyway, some more specific observations:
* ISTM that this is the wrong thing for shared BufFiles:
+BufFile * +BufFileImport(BufFileDescriptor *descriptor) +{
...
+ file->isInterXact = true; /* prevent cleanup by this backend */
There is only one user of isInterXact = true BufFiles at present,
tuplestore.c. It, in turn, only does so for cases that require
persistent tuple stores. A quick audit of these tuplestore.c callers
show this to just be cursor support code within portalmem.c. Here is
the relevant tuplestore_begin_heap() rule that that code adheres to,
unlike the code I've quoted above:
* interXact: if true, the files used for on-disk storage persist beyond the
* end of the current transaction. NOTE: It's the caller's responsibility to
* create such a tuplestore in a memory context and resource owner that will
* also survive transaction boundaries, and to ensure the tuplestore is closed
* when it's no longer wanted.
I don't think it's right for buffile.c to know anything about file
paths directly -- I'd say that that's a modularity violation.
PathNameOpenFile() is called by very few callers at the moment, all of
them very low level (e.g. md.c), but you're using it within buffile.c
to open a path to the file that you obtain from shared memory
directly. This is buggy because the following code won't be reached in
workers that call your BufFileImport() function:
/* Mark it for deletion at close */
VfdCache[file].fdstate |= FD_TEMPORARY;
/* Register it with the current resource owner */
if (!interXact)
{
VfdCache[file].fdstate |= FD_XACT_TEMPORARY;
ResourceOwnerEnlargeFiles(CurrentResourceOwner);
ResourceOwnerRememberFile(CurrentResourceOwner, file);
VfdCache[file].resowner = CurrentResourceOwner;
/* ensure cleanup happens at eoxact */
have_xact_temporary_files = true;
}
Certainly, you don't want the "Mark it for deletion at close" bit.
Deletion should not happen at eoxact for non-owners-but-sharers
(within FileClose()), but you *do* want CleanupTempFiles() to call
FileClose() for the virtual file descriptors you've opened in the
backend, to do some other cleanup. In general, you want to buy into
resource ownership for workers. As things stand, I think that this
will leak virtual file descriptors. That's really well hidden because
there is a similar CleanupTempFiles() call at proc exit, I think.
(Didn't take the time to make sure that that's what masked problems.
I'm sure that you want minimal divergence with serial cases,
resource-ownership-wise, in any case.)
Instead of all this, I suggest copying some of my changes to fd.c, so
that resource ownership within fd.c differentiates between a vfd that
is owned by the backend in the conventional sense, including having a
need to delete at eoxact, as well as a lesser form of ownership where
deletion should not happen. Maybe you'll end up using my BufFileUnify
interface [1]https://wiki.postgresql.org/wiki/Parallel_External_Sort#buffile.c.2C_and_BufFile_unification -- Peter Geoghegan within workers (instead of just within the leader, as
with parallel tuplesort), and have it handle all of that for you.
Currently, that would mean that there'd be an unused/0 sized "local"
segment for the unified BufFile, but I was thinking of making that not
happen unless and until a new segment is actually needed, so even that
minor wart wouldn't necessarily affect you.
Some assorted notes on the status: I need to do some thinking about
the file cleanup logic: both explicit deletes at the earliest possible
time, and failure/error paths. Currently the creator of each file is
responsible for cleaning it up, but I guess if the creator aborts
early the file disappears underneath the others' feet, and then I
guess they might raise a confusing error report that races against the
root cause error report; I'm looking into that. Rescans and skew
buckets not finished yet.
The rescan code path seems to segfault when the regression tests are
run. There is a NULL pointer dereference here:
@@ -985,6 +1855,14 @@ ExecReScanHashJoin(HashJoinState *node)
node->hj_HashTable = NULL;
node->hj_JoinState = HJ_BUILD_HASHTABLE;+ if (HashJoinTableIsShared(node->hj_HashTable)) + { + /* Coordinate a rewind to the shared hash table creation phase. */ + BarrierWaitSet(&hashNode->shared_table_data->barrier, + PHJ_PHASE_BEGINNING, + WAIT_EVENT_HASHJOIN_REWINDING3); + } +
Clearly, HashJoinTableIsShared() should not be called when its
argument (in this case node->hj_HashTable) is NULL.
In general, I think you should try to set expectations about what
happens when the regression tests run up front, because that's usually
the first thing reviewers do.
Various compiler warnings on my system:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHash.c:1376:7:
warning: variable ‘size_before_shrink’ set but not used
[-Wunused-but-set-variable]
Size size_before_shrink = 0;
^
...
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:
In function ‘ExecHashJoinCloseBatch’:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:1548:28:
warning: variable ‘participant’ set but not used
[-Wunused-but-set-variable]
HashJoinParticipantState *participant;
^
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:
In function ‘ExecHashJoinRewindBatches’:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:1587:23:
warning: variable ‘batch_reader’ set but not used
[-Wunused-but-set-variable]
HashJoinBatchReader *batch_reader;
^
Is this change really needed?:
--- a/src/backend/executor/nodeSeqscan.c +++ b/src/backend/executor/nodeSeqscan.c @@ -31,6 +31,8 @@ #include "executor/nodeSeqscan.h" #include "utils/rel.h"+#include <unistd.h> + static void InitScanRelation(SeqScanState *node, EState *estate, int eflags); static TupleTableSlot *SeqNext(SeqScanState *node);
That's all I have for now...
[1]: https://wiki.postgresql.org/wiki/Parallel_External_Sort#buffile.c.2C_and_BufFile_unification -- Peter Geoghegan
--
Peter Geoghegan
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Jan 10, 2017 at 8:56 PM, Peter Geoghegan <pg@heroku.com> wrote:
Instead of all this, I suggest copying some of my changes to fd.c, so
that resource ownership within fd.c differentiates between a vfd that
is owned by the backend in the conventional sense, including having a
need to delete at eoxact, as well as a lesser form of ownership where
deletion should not happen.
If multiple processes are using the same file via the BufFile
interface, I think that it is absolutely necessary that there should
be a provision to track the "attach count" of the BufFile. Each
process that reaches EOXact decrements the attach count and when it
reaches 0, the process that reduced it to 0 removes the BufFile. I
think anything that's based on the notion that leaders will remove
files and workers won't is going to be fragile and limiting, and I am
going to push hard against any such proposal.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jan 11, 2017 at 10:57 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, Jan 10, 2017 at 8:56 PM, Peter Geoghegan <pg@heroku.com> wrote:
Instead of all this, I suggest copying some of my changes to fd.c, so
that resource ownership within fd.c differentiates between a vfd that
is owned by the backend in the conventional sense, including having a
need to delete at eoxact, as well as a lesser form of ownership where
deletion should not happen.If multiple processes are using the same file via the BufFile
interface, I think that it is absolutely necessary that there should
be a provision to track the "attach count" of the BufFile. Each
process that reaches EOXact decrements the attach count and when it
reaches 0, the process that reduced it to 0 removes the BufFile. I
think anything that's based on the notion that leaders will remove
files and workers won't is going to be fragile and limiting, and I am
going to push hard against any such proposal.
Okay. My BufFile unification approach happens to assume that backends
clean up after themselves, but that isn't a ridged assumption (of
course, these are always temp files, so we reason about them as temp
files). It could be based on a refcount fairly easily, such that, as
you say here, deletion of files occurs within workers (that "own" the
files) only as a consequence of their being the last backend with a
reference, that must therefore "turn out the lights" (delete the
file). That seems consistent with what I've done within fd.c, and what
I suggested to Thomas (that he more or less follow that approach).
You'd probably still want to throw an error when workers ended up not
deleting BufFile segments they owned, though, at least for parallel
tuplesort.
This idea is something that's much more limited than the
SharedTemporaryFile() API that you sketched on the parallel sort
thread, because it only concerns resource management, and not how to
make access to the shared file concurrency safe in any special,
standard way. I think that this resource management is something that
should be managed by buffile.c (and the temp file routines within fd.c
that are morally owned by buffile.c, their only caller). It shouldn't
be necessary for a client of this new infrastructure, such as parallel
tuplesort or parallel hash join, to know anything about file paths.
Instead, they should be passing around some kind of minimal
private-to-buffile state in shared memory that coordinates backends
participating in BufFile unification. Private state created by
buffile.c, and passed back to buffile.c. Everything should be
encapsulated within buffile.c, IMV, making parallel implementations as
close as possible to their serial implementations.
--
Peter Geoghegan
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jan 11, 2017 at 2:20 PM, Peter Geoghegan <pg@heroku.com> wrote:
You'd probably still want to throw an error when workers ended up not
deleting BufFile segments they owned, though, at least for parallel
tuplesort.
Don't see why.
This idea is something that's much more limited than the
SharedTemporaryFile() API that you sketched on the parallel sort
thread, because it only concerns resource management, and not how to
make access to the shared file concurrency safe in any special,
standard way.
Actually, I only intended that sketch to be about resource management.
Sounds like I didn't explain very well.
Instead, they should be passing around some kind of minimal
private-to-buffile state in shared memory that coordinates backends
participating in BufFile unification. Private state created by
buffile.c, and passed back to buffile.c. Everything should be
encapsulated within buffile.c, IMV, making parallel implementations as
close as possible to their serial implementations.
That seems reasonable although I haven't studied the details carefully as yet.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jan 11, 2017 at 11:20 AM, Peter Geoghegan <pg@heroku.com> wrote:
If multiple processes are using the same file via the BufFile
interface, I think that it is absolutely necessary that there should
be a provision to track the "attach count" of the BufFile. Each
process that reaches EOXact decrements the attach count and when it
reaches 0, the process that reduced it to 0 removes the BufFile. I
think anything that's based on the notion that leaders will remove
files and workers won't is going to be fragile and limiting, and I am
going to push hard against any such proposal.Okay. My BufFile unification approach happens to assume that backends
clean up after themselves, but that isn't a ridged assumption (of
course, these are always temp files, so we reason about them as temp
files).
Also, to be clear, and to avoid confusion: I don't think anyone wants
an approach "that's based on the notion that leaders will remove files
and workers won't". All that has been suggested is that the backend
that creates the file should be responsible for deleting the file, by
definition. And, that any other backend that may have files owned by
another backend must be sure to not try to access them after the owner
deletes them. (Typically, that would be ensured by some barrier
condition, some dependency, inherent to how the parallel operation is
implemented.)
I will implement the reference count thing.
--
Peter Geoghegan
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jan 11, 2017 at 12:05 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Jan 11, 2017 at 2:20 PM, Peter Geoghegan <pg@heroku.com> wrote:
You'd probably still want to throw an error when workers ended up not
deleting BufFile segments they owned, though, at least for parallel
tuplesort.Don't see why.
Simply because that's not expected as things stand -- why should the
file go away in that context? (Admittedly, that doesn't seem like an
excellent reason now.)
I actually like the idea of a reference count, the more I think about
it, since it doesn't actually have any tension with my original idea
of ownership. If something like a randomAccess parallel tuplesort
leader merge needs to write new segments (which it almost certainly
*won't* anyway, due to my recent V7 changes), then it can still own
those new segments itself, alone, and delete them on its own in the
manner of conventional temp files, because we can still restrict the
shared refcount mechanism to the deletion of "initial" segments. The
refcount == 0 deleter only deletes those initial segments, and not any
same-BufFile segments that might have been added (added to append to
our unified BufFile within leader). I think that parallel hash join
won't use this at all, and, as I said, it's only a theoretical
requirement for parallel tuplesort, which will generally recycle
blocks from worker temp files for its own writes all the time for
randomAccess cases, the only cases that ever write within logtape.c.
So, the only BufFile shared state needed, that must be maintained over
time, is the refcount variable itself. The size of the "initial"
BufFile (from which we derive number of new segments during
unification) is passed, but it doesn't get maintained in shared
memory. BufFile size remains a one way, one time message needed during
unification. I only really need to tweak things in fd.c temp routines
to make all this work.
This is something I like because it makes certain theoretically useful
things easier. Say, for example, we wanted to have tuplesort workers
merge worker final materialized tapes (their final output), in order
to arrange for the leader to have fewer than $NWORKER runs to merge at
the end -- that's made easier by the refcount stuff. (I'm still not
convinced that that's actually going to make CREATE INDEX faster.
Still, it should, on general principle, be easy to write a patch that
makes it happen -- a good overall design should leave things so that
writing that prototype patch is easy.)
This idea is something that's much more limited than the
SharedTemporaryFile() API that you sketched on the parallel sort
thread, because it only concerns resource management, and not how to
make access to the shared file concurrency safe in any special,
standard way.Actually, I only intended that sketch to be about resource management.
Sounds like I didn't explain very well.
I'm glad to hear that, because I was very puzzled by what you said. I
guess I was thrown off by "shared read pointers". I don't want to get
into the business of flushing out dirty buffers, or making sure that
every backend stays in lockstep about what the total size of the
BufFile needs to be. It's so much simpler to just have clear
"barriers" for each parallel operation, where backends present a large
amount of immutable state to one other backend at the end, and tells
it how big its BufFile is only once. (It's not quite immutable, since
randomAccess recycle of temp files can happen within logtape.c, but
the point is that there should be very little back and forth -- that
needs to be severely restricted.)
--
Peter Geoghegan
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jan 11, 2017 at 2:56 PM, Peter Geoghegan <pg@heroku.com> wrote:
On Fri, Jan 6, 2017 at 12:01 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:Here is a new WIP patch. I have plenty of things to tidy up (see note
at end), but the main ideas are now pretty clear and I'd appreciate
some feedback.I have some review feedback for your V3. I've chosen to start with the
buffile.c stuff, since of course it might share something with my
parallel tuplesort patch. This isn't comprehensive, but I will have
more comprehensive feedback soon.
Thanks!
I'm not surprised that you've generally chosen to make shared BufFile
management as simple as possible, with no special infrastructure other
than the ability to hold open other backend temp files concurrently
within a worker, and no writing to another worker's temp file, or
shared read pointer. As you put it, everything is immutable. I
couldn't see much opportunity for adding a lot of infrastructure that
wasn't written explicitly as parallel hash join code/infrastructure.
My sense is that that was a good decision. I doubted that you'd ever
want some advanced, generic shared BufFile thing with multiple read
pointers, built-in cache coherency, etc. (Robert seemed to think that
you'd go that way, though.)
Right, this is extremely minimalist infrastructure. fd.c is
unchanged. buffile.c only gains the power to export/import read-only
views of BufFiles. There is no 'unification' of BufFiles: each hash
join participant simply reads from the buffile it wrote, and then
imports and reads from its peers' BufFiles, until all are exhausted;
so the 'unification' is happening in caller code which knows about the
set of participants and manages shared read positions. Clearly there
are some ownership/cleanup issues to straighten out, but I think those
problems are fixable (probably involving refcounts).
I'm entirely willing to throw that away and use the unified BufFile
concept, if it can be extended to support multiple readers of the
data, where every participant unifies the set of files. I have so far
assumed that it would be most efficient for each participant to read
from the file that it wrote before trying to read from files written
by other participants. I'm reading your patch now; more soon.
Anyway, some more specific observations:
* ISTM that this is the wrong thing for shared BufFiles:
+BufFile * +BufFileImport(BufFileDescriptor *descriptor) +{...
+ file->isInterXact = true; /* prevent cleanup by this backend */
There is only one user of isInterXact = true BufFiles at present,
tuplestore.c. It, in turn, only does so for cases that require
persistent tuple stores. A quick audit of these tuplestore.c callers
show this to just be cursor support code within portalmem.c. Here is
the relevant tuplestore_begin_heap() rule that that code adheres to,
unlike the code I've quoted above:* interXact: if true, the files used for on-disk storage persist beyond the
* end of the current transaction. NOTE: It's the caller's responsibility to
* create such a tuplestore in a memory context and resource owner that will
* also survive transaction boundaries, and to ensure the tuplestore is closed
* when it's no longer wanted.
Hmm. Yes, that is an entirely bogus use of isInterXact. I am
thinking about how to fix that with refcounts.
I don't think it's right for buffile.c to know anything about file
paths directly -- I'd say that that's a modularity violation.
PathNameOpenFile() is called by very few callers at the moment, all of
them very low level (e.g. md.c), but you're using it within buffile.c
to open a path to the file that you obtain from shared memory
Hmm. I'm not seeing the modularity violation. buffile.c uses
interfaces already exposed by fd.c to do this: OpenTemporaryFile,
then FilePathName to find the path, then PathNameOpenFile to open from
another process. I see that your approach instead has client code
provide more meta data so that things can be discovered, which may
well be a much better idea.
directly. This is buggy because the following code won't be reached in
workers that call your BufFileImport() function:/* Mark it for deletion at close */
VfdCache[file].fdstate |= FD_TEMPORARY;/* Register it with the current resource owner */
if (!interXact)
{
VfdCache[file].fdstate |= FD_XACT_TEMPORARY;ResourceOwnerEnlargeFiles(CurrentResourceOwner);
ResourceOwnerRememberFile(CurrentResourceOwner, file);
VfdCache[file].resowner = CurrentResourceOwner;/* ensure cleanup happens at eoxact */
have_xact_temporary_files = true;
}
Right, that is a problem. A refcount mode could fix that; virtual
file descriptors would be closed in every backend using the current
resource owner, and the files would be deleted when the last one turns
out the lights.
Certainly, you don't want the "Mark it for deletion at close" bit.
Deletion should not happen at eoxact for non-owners-but-sharers
(within FileClose()), but you *do* want CleanupTempFiles() to call
FileClose() for the virtual file descriptors you've opened in the
backend, to do some other cleanup. In general, you want to buy into
resource ownership for workers. As things stand, I think that this
will leak virtual file descriptors. That's really well hidden because
there is a similar CleanupTempFiles() call at proc exit, I think.
(Didn't take the time to make sure that that's what masked problems.
I'm sure that you want minimal divergence with serial cases,
resource-ownership-wise, in any case.)Instead of all this, I suggest copying some of my changes to fd.c, so
that resource ownership within fd.c differentiates between a vfd that
is owned by the backend in the conventional sense, including having a
need to delete at eoxact, as well as a lesser form of ownership where
deletion should not happen. Maybe you'll end up using my BufFileUnify
interface [1] within workers (instead of just within the leader, as
with parallel tuplesort), and have it handle all of that for you.
Currently, that would mean that there'd be an unused/0 sized "local"
segment for the unified BufFile, but I was thinking of making that not
happen unless and until a new segment is actually needed, so even that
minor wart wouldn't necessarily affect you.
Ok, I'm studying that code now.
Some assorted notes on the status: I need to do some thinking about
the file cleanup logic: both explicit deletes at the earliest possible
time, and failure/error paths. Currently the creator of each file is
responsible for cleaning it up, but I guess if the creator aborts
early the file disappears underneath the others' feet, and then I
guess they might raise a confusing error report that races against the
root cause error report; I'm looking into that. Rescans and skew
buckets not finished yet.The rescan code path seems to segfault when the regression tests are
run. There is a NULL pointer dereference here:@@ -985,6 +1855,14 @@ ExecReScanHashJoin(HashJoinState *node)
node->hj_HashTable = NULL;
node->hj_JoinState = HJ_BUILD_HASHTABLE;+ if (HashJoinTableIsShared(node->hj_HashTable)) + { + /* Coordinate a rewind to the shared hash table creation phase. */ + BarrierWaitSet(&hashNode->shared_table_data->barrier, + PHJ_PHASE_BEGINNING, + WAIT_EVENT_HASHJOIN_REWINDING3); + } +Clearly, HashJoinTableIsShared() should not be called when its
argument (in this case node->hj_HashTable) is NULL.In general, I think you should try to set expectations about what
happens when the regression tests run up front, because that's usually
the first thing reviewers do.
Apologies, poor form. That block can be commented out for now because
rescan support is obviously incomplete, and I didn't mean to post it
that way. Doing so reveals two remaining test failures: "join" and
"rowsecurity" managed to lose a couple of rows. Oops. I will figure
out what I broke and have a fix for that in my next version.
Various compiler warnings on my system:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHash.c:1376:7:
warning: variable ‘size_before_shrink’ set but not used
[-Wunused-but-set-variable]
Size size_before_shrink = 0;
^
In this case it was only used in dtrace builds; I will make sure any
such code is compiled out when in non-dtrace builds.
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:
In function ‘ExecHashJoinCloseBatch’:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:1548:28:
warning: variable ‘participant’ set but not used
[-Wunused-but-set-variable]
HashJoinParticipantState *participant;
^
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:
In function ‘ExecHashJoinRewindBatches’:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:1587:23:
warning: variable ‘batch_reader’ set but not used
[-Wunused-but-set-variable]
HashJoinBatchReader *batch_reader;
^Is this change really needed?:
--- a/src/backend/executor/nodeSeqscan.c +++ b/src/backend/executor/nodeSeqscan.c @@ -31,6 +31,8 @@ #include "executor/nodeSeqscan.h" #include "utils/rel.h"+#include <unistd.h> + static void InitScanRelation(SeqScanState *node, EState *estate, int eflags); static TupleTableSlot *SeqNext(SeqScanState *node);
Right, will clean up.
That's all I have for now...
Thanks! I'm away from my computer for a couple of days but will have
a new patch series early next week, and hope to have a better handle
on what's involved in adopting the 'unification' approach here
instead.
--
Thomas Munro
http://www.enterprisedb.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Jan 12, 2017 at 9:07 AM, Thomas Munro <thomas.munro@enterprisedb.com
wrote:
On Wed, Jan 11, 2017 at 2:56 PM, Peter Geoghegan <pg@heroku.com> wrote:
On Fri, Jan 6, 2017 at 12:01 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:Here is a new WIP patch. I have plenty of things to tidy up (see note
at end), but the main ideas are now pretty clear and I'd appreciate
some feedback.I have some review feedback for your V3. I've chosen to start with the
buffile.c stuff, since of course it might share something with my
parallel tuplesort patch. This isn't comprehensive, but I will have
more comprehensive feedback soon.Thanks!
I'm not surprised that you've generally chosen to make shared BufFile
management as simple as possible, with no special infrastructure other
than the ability to hold open other backend temp files concurrently
within a worker, and no writing to another worker's temp file, or
shared read pointer. As you put it, everything is immutable. I
couldn't see much opportunity for adding a lot of infrastructure that
wasn't written explicitly as parallel hash join code/infrastructure.
My sense is that that was a good decision. I doubted that you'd ever
want some advanced, generic shared BufFile thing with multiple read
pointers, built-in cache coherency, etc. (Robert seemed to think that
you'd go that way, though.)Right, this is extremely minimalist infrastructure. fd.c is
unchanged. buffile.c only gains the power to export/import read-only
views of BufFiles. There is no 'unification' of BufFiles: each hash
join participant simply reads from the buffile it wrote, and then
imports and reads from its peers' BufFiles, until all are exhausted;
so the 'unification' is happening in caller code which knows about the
set of participants and manages shared read positions. Clearly there
are some ownership/cleanup issues to straighten out, but I think those
problems are fixable (probably involving refcounts).I'm entirely willing to throw that away and use the unified BufFile
concept, if it can be extended to support multiple readers of the
data, where every participant unifies the set of files. I have so far
assumed that it would be most efficient for each participant to read
from the file that it wrote before trying to read from files written
by other participants. I'm reading your patch now; more soon.Anyway, some more specific observations:
* ISTM that this is the wrong thing for shared BufFiles:
+BufFile * +BufFileImport(BufFileDescriptor *descriptor) +{...
+ file->isInterXact = true; /* prevent cleanup by this backend */
There is only one user of isInterXact = true BufFiles at present,
tuplestore.c. It, in turn, only does so for cases that require
persistent tuple stores. A quick audit of these tuplestore.c callers
show this to just be cursor support code within portalmem.c. Here is
the relevant tuplestore_begin_heap() rule that that code adheres to,
unlike the code I've quoted above:* interXact: if true, the files used for on-disk storage persist beyond
the
* end of the current transaction. NOTE: It's the caller's
responsibility to
* create such a tuplestore in a memory context and resource owner that
will
* also survive transaction boundaries, and to ensure the tuplestore is
closed
* when it's no longer wanted.
Hmm. Yes, that is an entirely bogus use of isInterXact. I am
thinking about how to fix that with refcounts.I don't think it's right for buffile.c to know anything about file
paths directly -- I'd say that that's a modularity violation.
PathNameOpenFile() is called by very few callers at the moment, all of
them very low level (e.g. md.c), but you're using it within buffile.c
to open a path to the file that you obtain from shared memoryHmm. I'm not seeing the modularity violation. buffile.c uses
interfaces already exposed by fd.c to do this: OpenTemporaryFile,
then FilePathName to find the path, then PathNameOpenFile to open from
another process. I see that your approach instead has client code
provide more meta data so that things can be discovered, which may
well be a much better idea.directly. This is buggy because the following code won't be reached in
workers that call your BufFileImport() function:/* Mark it for deletion at close */
VfdCache[file].fdstate |= FD_TEMPORARY;/* Register it with the current resource owner */
if (!interXact)
{
VfdCache[file].fdstate |= FD_XACT_TEMPORARY;ResourceOwnerEnlargeFiles(CurrentResourceOwner);
ResourceOwnerRememberFile(CurrentResourceOwner, file);
VfdCache[file].resowner = CurrentResourceOwner;/* ensure cleanup happens at eoxact */
have_xact_temporary_files = true;
}Right, that is a problem. A refcount mode could fix that; virtual
file descriptors would be closed in every backend using the current
resource owner, and the files would be deleted when the last one turns
out the lights.Certainly, you don't want the "Mark it for deletion at close" bit.
Deletion should not happen at eoxact for non-owners-but-sharers
(within FileClose()), but you *do* want CleanupTempFiles() to call
FileClose() for the virtual file descriptors you've opened in the
backend, to do some other cleanup. In general, you want to buy into
resource ownership for workers. As things stand, I think that this
will leak virtual file descriptors. That's really well hidden because
there is a similar CleanupTempFiles() call at proc exit, I think.
(Didn't take the time to make sure that that's what masked problems.
I'm sure that you want minimal divergence with serial cases,
resource-ownership-wise, in any case.)Instead of all this, I suggest copying some of my changes to fd.c, so
that resource ownership within fd.c differentiates between a vfd that
is owned by the backend in the conventional sense, including having a
need to delete at eoxact, as well as a lesser form of ownership where
deletion should not happen. Maybe you'll end up using my BufFileUnify
interface [1] within workers (instead of just within the leader, as
with parallel tuplesort), and have it handle all of that for you.
Currently, that would mean that there'd be an unused/0 sized "local"
segment for the unified BufFile, but I was thinking of making that not
happen unless and until a new segment is actually needed, so even that
minor wart wouldn't necessarily affect you.Ok, I'm studying that code now.
Some assorted notes on the status: I need to do some thinking about
the file cleanup logic: both explicit deletes at the earliest possible
time, and failure/error paths. Currently the creator of each file is
responsible for cleaning it up, but I guess if the creator aborts
early the file disappears underneath the others' feet, and then I
guess they might raise a confusing error report that races against the
root cause error report; I'm looking into that. Rescans and skew
buckets not finished yet.The rescan code path seems to segfault when the regression tests are
run. There is a NULL pointer dereference here:@@ -985,6 +1855,14 @@ ExecReScanHashJoin(HashJoinState *node)
node->hj_HashTable = NULL;
node->hj_JoinState = HJ_BUILD_HASHTABLE;+ if (HashJoinTableIsShared(node->hj_HashTable)) + { + /* Coordinate a rewind to the shared hash tablecreation phase. */
+ BarrierWaitSet(&hashNode->shared_table_data->barrier, + PHJ_PHASE_BEGINNING, + WAIT_EVENT_HASHJOIN_REWINDING3); + } +Clearly, HashJoinTableIsShared() should not be called when its
argument (in this case node->hj_HashTable) is NULL.In general, I think you should try to set expectations about what
happens when the regression tests run up front, because that's usually
the first thing reviewers do.Apologies, poor form. That block can be commented out for now because
rescan support is obviously incomplete, and I didn't mean to post it
that way. Doing so reveals two remaining test failures: "join" and
"rowsecurity" managed to lose a couple of rows. Oops. I will figure
out what I broke and have a fix for that in my next version.Various compiler warnings on my system:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/
executor/nodeHash.c:1376:7:
warning: variable ‘size_before_shrink’ set but not used
[-Wunused-but-set-variable]
Size size_before_shrink = 0;
^In this case it was only used in dtrace builds; I will make sure any
such code is compiled out when in non-dtrace builds./home/pg/pgbuild/builds/root/../../postgresql/src/backend/
executor/nodeHashjoin.c:
In function ‘ExecHashJoinCloseBatch’:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:1548:28:
warning: variable ‘participant’ set but not used
[-Wunused-but-set-variable]
HashJoinParticipantState *participant;
^
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:
In function ‘ExecHashJoinRewindBatches’:
/home/pg/pgbuild/builds/root/../../postgresql/src/backend/executor/nodeHashjoin.c:1587:23:
warning: variable ‘batch_reader’ set but not used
[-Wunused-but-set-variable]
HashJoinBatchReader *batch_reader;
^Is this change really needed?:
--- a/src/backend/executor/nodeSeqscan.c +++ b/src/backend/executor/nodeSeqscan.c @@ -31,6 +31,8 @@ #include "executor/nodeSeqscan.h" #include "utils/rel.h"+#include <unistd.h> + static void InitScanRelation(SeqScanState *node, EState *estate, inteflags);
static TupleTableSlot *SeqNext(SeqScanState *node);
Right, will clean up.
That's all I have for now...
Thanks! I'm away from my computer for a couple of days but will have
a new patch series early next week, and hope to have a better handle
on what's involved in adopting the 'unification' approach here
instead.--
Thomas Munro
http://www.enterprisedb.com--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi Thomas,
I was trying to analyse the performance of TPC-H queries with your patch
and came across following results,
Q9 and Q21 were crashing, both of them had following bt in core dump (I
thought it might be helpful),
#0 0x0000000010757da4 in pfree (pointer=0x3fff78d11000) at mcxt.c:1012
#1 0x000000001032c574 in ExecHashIncreaseNumBatches
(hashtable=0x1003af6da60) at nodeHash.c:1124
#2 0x000000001032d518 in ExecHashTableInsert (hashtable=0x1003af6da60,
slot=0x1003af695c0, hashvalue=2904801109, preload=1 '\001') at
nodeHash.c:1700
#3 0x0000000010330fd4 in ExecHashJoinPreloadNextBatch
(hjstate=0x1003af39118) at nodeHashjoin.c:886
#4 0x00000000103301fc in ExecHashJoin (node=0x1003af39118) at
nodeHashjoin.c:376
#5 0x0000000010308644 in ExecProcNode (node=0x1003af39118) at
execProcnode.c:490
#6 0x000000001031f530 in fetch_input_tuple (aggstate=0x1003af38910) at
nodeAgg.c:587
#7 0x0000000010322b50 in agg_fill_hash_table (aggstate=0x1003af38910) at
nodeAgg.c:2304
#8 0x000000001032239c in ExecAgg (node=0x1003af38910) at nodeAgg.c:1942
#9 0x0000000010308694 in ExecProcNode (node=0x1003af38910) at
execProcnode.c:509
#10 0x0000000010302a1c in ExecutePlan (estate=0x1003af37fa0,
planstate=0x1003af38910, use_parallel_mode=0 '\000', operation=CMD_SELECT,
sendTuples=1 '\001', numberTuples=0,
direction=ForwardScanDirection, dest=0x1003af19390) at execMain.c:1587
In case you want to know, I was using TPC-H with 20 scale factor. Please
let me know if you want anymore information on this.
--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/
On Wed, Jan 11, 2017 at 7:37 PM, Thomas Munro
<thomas.munro@enterprisedb.com> wrote:
Hmm. Yes, that is an entirely bogus use of isInterXact. I am
thinking about how to fix that with refcounts.
Cool. As I said, the way I'd introduce refcounts would not be very
different from what I've already done -- there'd still be a strong
adherence to the use of resource managers to clean-up, with that
including exactly one particular backend doing the extra step of
deletion. The refcount only changes which backend does that extra step
in corner cases, which is conceptually a very minor change.
I don't think it's right for buffile.c to know anything about file
paths directly -- I'd say that that's a modularity violation.
PathNameOpenFile() is called by very few callers at the moment, all of
them very low level (e.g. md.c), but you're using it within buffile.c
to open a path to the file that you obtain from shared memoryHmm. I'm not seeing the modularity violation. buffile.c uses
interfaces already exposed by fd.c to do this: OpenTemporaryFile,
then FilePathName to find the path, then PathNameOpenFile to open from
another process. I see that your approach instead has client code
provide more meta data so that things can be discovered, which may
well be a much better idea.
Indeed, my point was that the metadata thing would IMV be better.
buffile.c shouldn't need to know about file paths, etc. Instead,
caller should pass BufFileImport()/BufFileUnify() simple private state
sufficient for routine to discover all details itself, based on a
deterministic scheme. In my tuplesort patch, that piece of state is:
/*
+ * BufFileOp is an identifier for a particular parallel operation involving
+ * temporary files. Parallel temp file operations must be discoverable across
+ * processes based on these details.
+ *
+ * These fields should be set by BufFileGetIdent() within leader process.
+ * Identifier BufFileOp makes temp files from workers discoverable within
+ * leader.
+ */
+typedef struct BufFileOp
+{
+ /*
+ * leaderPid is leader process PID.
+ *
+ * tempFileIdent is an identifier for a particular temp file (or parallel
+ * temp file op) for the leader. Needed to distinguish multiple parallel
+ * temp file operations within a given leader process.
+ */
+ int leaderPid;
+ long tempFileIdent;
+} BufFileOp;
+
Right, that is a problem. A refcount mode could fix that; virtual
file descriptors would be closed in every backend using the current
resource owner, and the files would be deleted when the last one turns
out the lights.
Yeah. That's basically what the BufFile unification process can
provide you with (or will, once I get around to implementing the
refcount thing, which shouldn't be too hard). As already noted, I'll
also want to make it defer creation of a leader-owned segment, unless
and until that proves necessary, which it never will for hash join.
Perhaps I should make superficial changes to unification in my patch
to suit your work, like rename the field BufFileOp.leaderPid to
BufFileOp.ownerPid, without actually changing any behaviors, except as
noted in the last paragraph. Since you only require that backends be
able to open up some other backend's temp file themselves for a short
while, that gives you everything you need. You'll be doing unification
in backends, and not just within the leader as in the tuplesort patch,
I believe, but that's just fine. All that matters is that you present
all data at once to a consuming backend via unification (since you
treat temp file contents as immutable, this will be true for hash
join, just as it is for tuplesort).
There is a good argument against my making such a tweak, however,
which is that maybe it's clearer to DBAs what's going on if temp file
names have the leader PID in them for all operations. So, maybe
BufFileOp.leaderPid isn't renamed to BufFileOp.ownerPid by me;
instead, you always make it the leader pid, while at the same time
having the leader dole out BufFileOp.tempFileIdent identifiers to each
worker as needed (see how I generate BufFileOps for an idea of what I
mean if it's not immediately clear). That's also an easy change, or at
least will be once the refcount thing is added.
--
Peter Geoghegan
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Jan 13, 2017 at 2:36 PM, Peter Geoghegan <pg@heroku.com> wrote:
[...]
Yeah. That's basically what the BufFile unification process can
provide you with (or will, once I get around to implementing the
refcount thing, which shouldn't be too hard). As already noted, I'll
also want to make it defer creation of a leader-owned segment, unless
and until that proves necessary, which it never will for hash join.
Hi Peter,
I have broken this up into a patch series, harmonised the private vs
shared hash table code paths better and fixed many things including
the problems with rescans and regression tests mentioned upthread.
You'll see that one of the patches is that throwaway BufFile
import/export facility, which I'll replace with your code as
discussed.
The three 'refactor' patches change the existing hash join code to
work in terms of chunks in more places. These may be improvements in
their own right, but mainly they pave the way for parallelism. The
later patches introduce single-batch and then multi-batch shared
tables.
The patches in the attached tarball are:
0001-nail-down-regression-test-row-order-v4.patch:
A couple of regression tests would fail with later refactoring that
changes the order of unmatched rows emitted by hash joins. So first,
let's fix that by adding ORDER BY in those places, without any code
changes.
0002-hj-add-dtrace-probes-v4.patch:
Before making any code changes, let's add some dtrace probes so that
we can measure time spent doing different phases of hash join work
before and after the later changes. The main problem with the probes
as I have them here (and the extra probes inserted by later patches in
the series) is that interesting query plans contain multiple hash
joins so these get all mixed up when you're trying to measure stuff,
so perhaps I should pass executor node IDs into all the probes. More
on this later. (If people don't want dtrace probes in the executor,
I'm happy to omit them and maintain that kind of thing locally for my
own testing purposes...)
0003-hj-refactor-memory-accounting-v4.patch:
Modify the existing hash join code to work in terms of chunks when
estimating and later tracking memory usage. This is probably more
accurate than the current tuple-based approach, because it tries to
take into account the space used by chunk headers and the wasted space
in chunks. In practice the difference is probably small, but it's
arguably more accurate; I did this because I need chunk-based
accounting the later patches. Also, make HASH_CHUNK_SIZE the actual
size of allocated chunks (ie the header information is included in
that size so we allocate exactly 32KB, not 32KB + a bit, for the
benefit of the dsa allocator which otherwise finishes up allocating
36KB).
0004-hj-refactor-batch-increases-v4.patch:
Modify the existing hash join code to detect work_mem exhaustion at
the point where chunks are allocated, instead of checking after every
tuple insertion. This matches the logic used for estimating, and more
importantly allows for some parallelism in later patches.
0005-hj-refactor-unmatched-v4.patch:
Modifies the existing hash join code to handle unmatched tuples in
right/full joins chunk-by-chunk. This is probably a cache-friendlier
scan order anyway, but the real goal is to provide a natural grain for
parallelism in a later patch.
0006-hj-barrier-v4.patch:
The patch from a nearby thread previously presented as a dependency of
this project. It might as well be considered part of this patch
series.
0007-hj-exec-detach-node-v4.patch
By the time ExecEndNode() runs in workers, ExecShutdownNode() has
already run. That's done on purpose because, for example, the hash
table needs to survive longer than the parallel environment to allow
EXPLAIN to peek at it. But it means that the Gather node has thrown
out the shared memory before any parallel-aware node below it gets to
run its Shutdown and End methods. So I invented ExecDetachNode()
which runs before ExecShutdownNode(), giving parallel-aware nodes a
chance to say goodbye before their shared memory vanishes. Better
ideas?
0008-hj-shared-single-batch-v4.patch:
Introduces hash joins with "Shared Hash" and "Parallel Shared Hash"
nodes, for single-batch joins only. If the planner has a partial
inner plan, it'll pick a Parallel Shared Hash plan to divide that over
K participants. Failing that, if the planner has a parallel-safe
inner plan and thinks that it can avoid batching by using work_mem * K
memory (shared by all K participants), it will now use a Shared Hash.
Otherwise it'll typically use a Hash plan as before. Without the
later patches, it will blow through work_mem * K if it turns out to
have underestimated the hash table size, because it lacks
infrastructure for dealing with batches.
The trickiest thing at this point in the series is that participants
(workers and the leader) can show up at any time, so there are three
places that provide synchronisation with a parallel hash join that is
already in progress. Those can be seen in ExecHashTableCreate,
MultiExecHash and ExecHashJoin (HJ_BUILD_HASHTABLE case).
0009-hj-shared-buffile-strawman-v4.patch:
Simple code for sharing BufFiles between backends. This is standing
in for Peter G's BufFile sharing facility with refcount-based cleanup.
0010-hj-shared-multi-batch-v4.patch:
Adds support for multi-batch joins with shared hash tables. At this
point, more complications appear: deadlock avoidance with the leader,
batch file sharing and coordinated batch number increases (shrinking
the hash table) while building or loading.
Some thoughts:
* Although this patch series adds a ton of wait points, in the common
case of a single batch inner join there is effectively only one:
participants wait for PHJ_PHASE_BUILDING to end and PHJ_PHASE_PROBING
to begin (resizing the hash table in between if necessary). For a
single batch outer join, there is one more wait point: participants
wait for PHJ_PHASE_PROBING to end so that PHJ_PHASE_UNMATCHED can
begin. The length of the wait for PHJ_PHASE_BUILDING to finish is
limited by the grain of the scattered data being loaded into the hash
table: if the source of parallelism is Parallel Seq Scan, then the
worst case scenario is that you run out of tuples to insert and
twiddle your thumbs while some other participant chews on the final
pageful of tuples. The wait for PHJ_PHASE_UNMATCHED (if applicable)
is similarly limited by the time it takes for the slowest participant
to scan the match bits of one chunk of tuples. All other phases and
associated wait points relate to multi-batch joins: either running out
of work_mem and needing to shrink the hash table, or coordinating
loading and various batches; in other words, ugly synchronisation only
enters the picture at the point where hash join starts doing IO
because you don't have enough work_mem.
* I wrestled with rescans for a long time; I think I have it right
now! The key thing to understand is that only the leader runs
ExecHashJoinReScan; new workers will be created for the next scan, so
the leader is able to get the barrier into the right state (attached
and fast-forwarded to PHJ_PHASE_PROBING if reusing the hash table,
detached and in the initial phase PHJ_PHASE_BEGINNING if we need to
recreate it).
* Skew table not supported yet.
* I removed the support for preloading data for the next batch; it
didn't seem to buy anything (it faithfully used up exactly all of your
work_mem for a brief moment, but since probing usually finishes very
close together in all participants anyway, no total execution time
seems to be saved) and added some complexity to the code; might be
worth revisiting but I'm not hopeful.
* The thing where different backends attach at different phases of the
hash join obviously creates a fairly large bug surface; of course we
can review the code and convince ourselves that it is correct, but
what is really needed is a test with 100% coverage that somehow
arranges for a worker to join at phases 0 to 12, and then perhaps also
for the leader to do the same; I have an idea for how to do that with
a debug build, more soon.
* Some of this needs to be more beautiful.
* With the patches up to 0008-hj-shared-single-batch.patch, I find
that typically I can get up to 3x or 4x speedups on queries like TPCH
Q9 that can benefit from a partial inner plan using Parallel Shared
Hash when work_mem is set 'just right', and at least some speedup on
queries without a partial inner plan but where the extra usable memory
available to Shared Hash can avoid the need to batching. (The best
cases I've seen probably combine these factors: avoiding batching and
dividing work up).
* With the full patch series up to 0010-hj-shared-multi-batch.patch,
it produces some terrible plans for some TPCH queries right now, and
I'm investigating that. Up to this point I have been focused on
getting the multi-batch code to work correctly, but will now turn some
attention to planning and efficiency and figure out what's happening
there.
--
Thomas Munro
http://www.enterprisedb.com
Attachments:
parallel-shared-hash-v4.tgzapplication/x-gzip; name=parallel-shared-hash-v4.tgzDownload
� ���X ���[��0��
E���-��v�����<&��8���y����XRk�-c�����Yj�M��M����]]��S��~����L�I5����z�E7�/���������~�#�g�~��m���D�Uo7��ns_���z����cN"��2�����t9[��d����=�C��7�7����Fu���(��U����� �Uc?�����.F�B����xxS0����n����nvy���V�%��ng��D�� ����?
�cQ�^�����p�w��{��:���?��)f�p��������*F�����k��aw�`�)���ju������z����+���a�rPe��-������b.gqiwG��T���sCx�m!;�x�L��aG"�w��P_�R�`�������W1����Y���� ���~n�����nGDQ��;�U�F�3���sS���k?R���?�]�U|8;>O���O�{.����Y�q��K!.�������4���������&���&�oN����(�`�;r���Q��D#�-����}������=��v9������T���
�i����r����7�:��,.�E�C�/�{��y���� `o��k������\n�6&�lfVS��,�nl�t�+��O������+?��
������QS���������H�W>f<[�Yr�����kw��gk6�%<h �f���.���4@Fg�uH��� +?��~��}�u�(q�=�9x� ��m4�}Z�I^���i��<�
��7�J��������������p�mD���U���5� �H���Lm#b����
~����w��X���[����� +�cb����?�:+����o+�m��+X5� �������5i��+�WP�������+�m�o8��A|�������o+�m��+Xy�7���9��xt�;�1%��x�M�m����<�B�A+p�������:���4�:��� ��D��o�s�U�m������
}��#Oox���-o0����/����r�w+�m����i���ZM��y����m��m��a�=�Tp=�����E���'>�?;=���7��s1�P�w�����w!��_���S�;��/�g�����e������_��Co6c1�'~������?)�} �@B\��~[~z%�yy*�=�x�����_�_���b���h�z�;��\�]����;1
���?��7�����0�?�?5�Cp����H�C�y�]~�n�����������`TQ�!���9���&���h�-��p��+�4��XB�C�J�����Sv�*��Pi��L��r�� �0F��&&`-�L��D�p6��@����|nO�����"�MG���H4�NS��z�r���Y8 ��m����������'����NV�N�a:)?*��Vk6W�4����aH�u��(kz0t������5D��<����������EO��m���I��^0��m�HR$���-$��Pv��FSJ�)�..�_<b9�n�<�L�$�c�A�5�����8\���L|�% gQ��Il��B��������{�2�D��M-���Vt�IMYII#���7��������CoN��)�Cb
�8D�&�������j�Mp��B�]��&O��|&�xq��O��Li�cs�5��S|����',�p���C(a��=��9�R����������q����_��>�.E�����W��!Yy&��R���xv^��v�V�y������dqW�{�����?��:��Qg?�����9m�5:��J�.��K��.f��A�����
�%�i�{M!�}u�3oP?z9A�+�:nT�@t[B(��D3F����})�4������';*��dS<�x'�O�hc��7��#��8���������{�F~�8}w|�W�}��6���-^��/$�#��\B�H8�q�_����+��mVo~�z�Qu$a7���E8��u~�����$;n�o�So���k�K]go�������������,��W��R��/1o�9l�G���������>��U�
��V��e��}ymm��
f��r$��i
��T�z7O��K)F{���x��r�O�xYN��#���$��co����K���!�]'/�-'q�������?�1���d�bn����H�{����m0�I~)^�'R�f#ukH"����BsK���t.%�z;��.o]]����_^}w����Y���������g���W�W% �[R���#����[�D��/���"��������7�r�\_k��mm��q���q{�+Z[��h�,�~���e��_�?��J;"�;�)$4%�����IPc1����YJ�W?�?�.d�'�����:�7; <�%[��1f��DV)����x��q0�&;��~���� ���P
n����"�G+������f�pv��5fi�m{d�r��7|����,g�E8
����{<��"��o/N��G�J���I���g`(�n����>S��K��`� >
��o���[��_������%�F7�*)����;�P��Ox����#e��EP6[���CrK�f�AE��I���h.���ri�Az]y�xz��^�������eI�]uJ?e7e90��W8M�I�,�����=�\�6���'_cF��!5�u��G\�x!�%K��G2�=h�Q����h����SV��?�DLl|���7x��"C*H��~1����/ #�.�J��Y�� ��)���G�f:�����,����yD�c"Z��:�B��j5��R3 ��� ����eB�`�������N�;��D*u) ��&Gtk�\NQ�"�\���[��r9y�M���Fy���3�Dk�R��TH g!����(����shj��w����wG����R����� y����W�������KO`�Q���s\�H�^b���u_H$a�����fO*�J��� T#���c��0mg��h��r� )����V�������+�G� C-F�)0������[����~�`|�����PM���Q3��lv +q���
�a�g�)s��K�5�
�v�~4 ����/�W�����A�������Y"?�!��]���d~�c��W@���9�{��_LL$Z��e g�e�
�}��|��PR���A����/�'];������f�;�2�����p��K��| ������x�� �m�E1B��������${|����K��m������W�N��s��H^�?�gk\nI�����q�E\�-�t��=�/�������Y?,�9����p�q6EFc�w�����*!��~���O������<$z�z}���]�7I3v�\ |<�|�-&���JK<�%6[]C# ��&K����f��.��c>4J��B^��X�����q��q�4/�x����7�|^ ��&-����;\��/��;d�'�u�Q�W+�#M��S<�c����/�`^"zT���i�e���+I�� wX�Y�T�(�5��b��4=�[���M�Tk��h���O�:���iEy% �-^oN@����1[�2�'�[�J)J�����k�����Ns�G��~�}�J���y�a�V;h������h�����D�@t`���:�w_0���(����7�)�-�~�1�V*`��T���F���A((QK�J�5]����>^��c���b7��I�Wl%��{NM(�t��v,w,�@
���5&���M�)F|�Q�.O���~��U�������U���F�#����x�/����")�g�d��j����c���!n8����v:�,���f){�Wk��5�
�?-��,��7�lh�T�Uo��R�[���������N{��������m�?d���<�����oe�iH��� +����2�Wi/�����-p�U��;�s�*���@����dQ�<��*I��������\�����{��������9�<��Hv�X���eJ����>��ORC��A���!������� ."�E�x��|��G�9,�(�`w�/�Q��D�����q8GSF+GO�Zy�>xPG�~'afAv�\�|!HE�r��������$�Et����??_N&QBn�'.��.�`.�k8����a��w��X����w����-��R
.�9"70��I�w��
�%I�*�������E�C�~Bu�`.XyB�.� Q:����I��3VE�!�yJJ���k$t��m��h�*M�
<� ����
J��(\��fJy����`��N/#�TQ����?Hy�V�MU�^ ]�������E@3bT���q��6H��a��eP!k���6H{�>� "�k��V%���tv'�D��_�HJ�
�Y0_�wx&B�:�=c��d� �5l��C�A[}�Bd���&�Q�]0Rk�i;�vj�����K�����.��P���&L
g%@!� ������j�|�x5���������_������g���������[\�Jgw�}�X���i{,���l���]�/O�e 'R�jd��6{��^_��l[�R�`���Z�Fp*��e;"��q�U�
SR�;^E�{�{f�0WQ�[����F�=�Jv@��6�p����$�X�&~�~)/���Q6���5�A#��sW��@k��5H�X�[)_��[�9��8B�m���R�o�krO���h*Y��x��_qZ<���
)�,�����4������s�����7A�L*��~�����C���M������Z�e��z_'�,����(�P9D��A]9��O-�3����f$A�Kp��b��U��c�'�dHA{�O�fK��x��fr,b�)k�jrr�3��z�����v�lV-Q>��� �����*s*���)�n%%��%)����.7��6C�����!�2�E+d���������p*����S<�����|�X\��������:B&n22�t�6��i4��yJ�5}{S���^�����\��R�L���Rc�AN���� ���/P�{��3N��{B��E���W��cgE�:xx�E���K%n�n4/��c��U~�sK��KsS�!��SEe%3+y���@o5Pk�w��f�~PWFq�Xw9�\��9�S�k�� /��|0A1~��"�������sw�������q�8&y���/O����k �� H��������$�����XT�bu�\��(�����}����!�6�,��>�<�7���i����?��w;L�BJ����F� =c,�
�:Z(����]��j�}o8�{^Z�Z��������A\�w�ws�*@�6�E�o��F�&����(�|{b#���ZMy4�������vzo��8�;��i�~+%���6:sf�M0\��Z������c�AjXU��Fa@+�?�n7Y��S��-��W��m�?d�������uX���0�O��^���������im�i�4e�������~/$������Sr9l���^v7d�������c)��~���JqE���s�/��o[�]#�P�8��!:��-X�S�Q U�aR-�0���^[�"o�#��Ki��qK@}<��i���w�f��sp�A�*��x�}Ev�hO�X������^sgg�p*:�<j����������U�������Ad��f7�-W*��7�X�OR"��#X����\���:�U���Ib���:������j��������� �
��7��8_ ��P���u�}+�;��JJ���'q��$��n=�L#�r�C��#��y��<��'�:,�6s��;no\�cq�V[q�f��6Q����K���Vi�L��R�UY����h3���U��5 �2���Ps����Te����Vt�A��'H<o�Z�h��I.�d��VXa��!��Bh#�1%�>M�������s���HG�m& �/�(&/�O>�Z �6���p�A� �Z�
�t�_���W� (g ��;���Rp��������8A�WR�U�T�Y��{!q����d�1�C���
��=�OXp6�'�_�`>Vj�gp�?d���Br�8���N��q��O������Q�t�=Kr���� 4�B���'�O�)b������7��+%��'�C�yL���&"lj�#��,��������p��Rgi���R(#��<^p\�~�"��]����v�p�����cps���)�@e���f��d�2�S���Bs���m����������,40(��u�g�{�7i��}uM}0�p�=m���o�{ ��)}+G?����n������t�;�����\���7D�1<���@��q�:�`�@+���~'��k�����~��D��j�Z��5���7��np>Y����X��][�����7�o!��e7G"�~��?E`��@�����]�����Wi�_/'�B�
�~�$���%]��=D����?/�%f+31�y�|i��9S��K�Of���x����q���o�����%y?|I`K����S�\�d�+(���>I��p��?JFi�tb ���p��C��z)�>pu�J�z������ �����'1���1&���~U����7R"���.z��#y�����TY�F��He-
�����]� 5��0!�H�]��Y��&��31���[��h�Q��J�pV�Q�s5|��N�e<�_�@t�k��x>��v6J��J�c�So�IY��X�D�:�!Q]'�N������@rX�������&�G��� W���A�#
�r��6������ �N�O�o�7�B��}�*!�-Z�v`���u6�GyY4W���
�.�������C��������D �������X����7�]�a��k�G��'$�[!��b���rK��+�Z���_��
�]$5�
"����=�]"]���9��l)6Bjw ��v��a&����u<�l �L����=p��KH-b>����f+MQ�&{M�c=��&8�
65�������k�G�t5���a0��o(�"�>�n�KE�`*����������kJ��!,f�Pv)D��X���~�B�HXL�z
0��M�x����Z��8������yM2�C
slbR�����
)M����
&9���1���nm����a�2�P����m���G�[N�?���o!d���� :��;I�
��hS8��O �0��M���G��Cn��v��L��V9�����'k%��3D��D�S�v�;}I^4Cf����K�[����K��da��U}��'��\R<��
�"�<#��H Q����*}E����?���>��Cy���Tw�}3���Awaa�'u���K�Yw$Q\�o�%t�bG��j��������v�����,k���?l
2JP<������S#����8d0�%y�5s���ZK�U`f��u����:��S�e��qb�P�� O�����JqCz�I �H&c"�N�����dsf� ��D�4�������]�� �d����I���o������x�E�/�����+��6]�������t~����r���:���{�����j��&L:�N�u���G���Y�>���
8��A�������Q�����/��\<��R��V�S��*e��W�[��4����I8�&������.e�h���I-�_��8��@��`R�P��Qp=������������]�n��Q/��!
�K�L}�E�$��O�_�z���l��(��:t6����������d���b�4�Z7�Bt�����}��@���_�f��d�� G����41��v�V�?������H��IV����x7�,�Y�wL8��\~W�� )rO'��n\������zc�">PV6�:���k����H�w�p9��_W ���+����T</r��|�d%�] ���,����3H��>@W�jr��0��F2 �d����
�9���X���!F�
DzpJ!�A�;�")�zb~���{����%I>�PX��$}�B���A|3=R� /w��/T����e�7[p��w~u���+����F5��Q�k����jj`5hq��2�:�h��7��?��
�#"���c�}����o��A'�[/ ��c�0����o�{��$D����W���#��7�n�����"�`e>R�q��'�?}��S�#��%�/�)��?��ki����#�����]
� ���o�`b��<�������S�5��.cVL�GX_8}9���.k����OOB�����������*������Oz���[:e�9<�X����Mh�`9��.��,&4m�N���2�T�j�7#(������G�q{f�
^�38��n s�Y�:�5��
����+b~
�>�����-$)�}I^A��xUW1�/P�-B�n> �K��`6!�2�����y9��{C�9�\l
�=�AW?�������E������]�1+�0ZMh =y�A i�"��q<���7�r���� ���o�~G��J��Ri����N _�����a(/s�C���[��h%��+��/j/1.D�N��D,Az,-�A��S�fJu��s�d?���K�w���F>��?� :R-$P��W9��������/.N{�O��*��
�|^?��G��� �{!�����$ ��TzX��yA�u���z�I����� Z1�R�#`i
����������3�����Z������!i-����C�P��n�#�$O<�r�$��fz�)��jz#�]�&�[+��
[����lL*Q�B�_��4)��vx�����/5_p��B�Ux9����I'
�� ���i������m%7���3�b��*���)�� d��P�;��������"u- Q���f�t�_w�{���E���� ����c.#`�����6]��]��y�64�{��}�qY����N����*5s{C��m�0��,��5p���|� �A�#q��P�4������,P��[@4����M:�F��t�I6���(H��4d�����'$,� :�Uj��2:�WW�'oK�6����/��.|�Q���5,4�I�7�JE�h� #�`GP��m�+Y�K@4Q���$����h�]W[+X��w��K�>�-��#y�"CIcp���-/o3�\,�
������M77uu�����4V�y`9g��������0d���r
*�5��������g�zj|6�89it�5Yfpgp�&Y>�����W_��$9����$2��1M��M��\6"���U������x���4��$��'��*����Xm�z����� ��(�$s0�vR >u�����G�s����s�p��]����)*��
T�<�����P�$�ys� io���F�0O!#�4Y��8��r1A]�1s�A
���X�:%�q���f����/�Y����j�y�l=na�6�L������0p&�}d����r�'�#,��{�u����w�_����PX/��G�V5Qxg����YM����ftJY��w�����w��D#�5C�*�p�%�[�q��D�[.�����r����I��-��k��V�s����{�{��7��������������#k���)v��I�f=�W���A�2I.�/��2$�K�C��>1�*)�O8�0.�x� ��8����=�%OCy��e�L\���)/Su}���&�<�H���V�4��0�����J��^�Z�t)EH�@D��C����,|fimS��HO�5,�A�W)��(�~�-K��}�\�@�1
����XG
�� �X�?|�Bl��L�\��u����Q������y�������0�U���X���`��_�V3�+��[��_����_���'�]n��i��{��:�f�����A�`��_����_�ek���Sg��������z��S��
p�Z�O#(�S���������
@t��������&o�p��{DE�`���h�+��
/&'l������l�V
�"���|X�C�s��V� �;�n���t;�a� ������Z��^�N���������
�en�.�@�u��C�,��y� n�����C�x���Q�������r����?�a�q��#�L�����C&M��L�*;)N�z�����_�bs�ZY���H��*YN��[���3�S�+��h��dS��$~���~q�V���x��R��� ��8�;�
��� ����C|�;U�5���Kn�9j�:�s<O�����5��/�����:������0:���5��>
��Vp ����?�Z1am9��5�
�2Y��!��c�
P
�(5��3��X�3��]];o�/t.����`�V���X�v(dF{�����?�_�w �?���zS���k�:��~*�K����k��e0��=��u��a��_����^w��k5��^���g+2�Hs�����1�*�T��A�d�;M�w�+'����pf/@��?��� ~�d��m�
���c�i�;x��� G����[u����R��0Q�M�q�m��������S���k����$C����1mX�V�V��~�1���Zm�ow:^F���~V���%��=��r���hk2��'���R9�� ��x#y��z�n �P*?��wu�����jS���Y&��r��2Z}�5!��q� �*����&@��Ec)MD��K�w�c�G�0����\�����*�N���������vgL�����K"������B�n�r����}�.�Ys?[lr��Zb��@'���`PZ � ��g&� %�4�0$�+���t�~6��D?�����^�[<���M�}��k�����O��-�7�(���$����#�r2��
3}��������R?^g+������j����7�UZ�&������ 06��x�a��f����w/�h�&Q���JXH�LC/�&��;gJ+&�(&�L��w��/��n�V�x�0kv����/�rs��i /��4H^��_3��cE_�d�D:���B/���|i����E��m���2$�~�)d�AY�#�
�I�M���'�M1�����~Kl4�AtNT�L�_���c����9�����(wZ�f�}5��| �p�I��L�����$y�G^����k��T0����)���?��fj�
��'C�0�C��8,�_��BgAc9����X�W����z�U;������R��0�ina���C}B�M�����p�q0���$� �b��F%���]�N*Z��$��ZR����<Df�-xAvmJx�N�[l,�\�e��,/h�`?,�Q#8�-�&�;Z����d�Gc��yT3-w��0.���t:k�0\�
C���k�����&��u��� 9V����<�H�a%��2_�����6�->�S��COT�����\��}��� ��I�b}/���!�Ud����g1u"g.���px����Q?���Cw�E�HS� �04]N�UX�qZ��N.|N������%t�*���}9��?�4�80('�G
��o����K���;=??=��X��]7�W#�&�"{'7M0�%��[aJ[��[��H�5����tX��P�Z�u^�,n6�b1x�NPj;�h�����w~E����;�z����i�l�99 ��(�]��[o���o^'A�����
���7�^J�aq��I�9�����J�sR�~��^�.O���� �<�2'L��Lo��X^*�Bq�������� ��dX��Tex��pm`w�����?�p"����(e����$�P�{6����5=�i���I}<��� �)#����:����m��D��jVG�W�@UJM��Y�)3q��4�c�q�'j P 0������$����K�3`s�J�D�W���Mh�!c�r���7��XJ+�;�gX5g8������;]�.�|��z�
\1�vTY�Dqx,5ue��N��E���&��u��^5����0����qL���"�;
�u9'w[7��Y<�����������d�5Yp�{'>����C���k�@CdQr��bI���H�:����W��y�s��&��K����m�B���R���&���3�����-T7j���R^�p�YThDHX��[<�)�Kb����F��S`�� ��Q��A���*���\x7x�
��<_�����5�D"`T��BP�#��b�E �����3��r(����3��N���>�������x��[�
�=$J�1���w!�~�}�J���,|�2�B}�cU����z\*Qx3������� �=���J��l2�fU�-Y�Xk���D���Q��QE�c:��U[U �\kl}[�������v����p���l"��`��RhD>��!C�Ry����x����(�<K%8UYR2Q�T���{d0%Y��&^�TU��@R��Y�@W�|�m�z�.: /���W���z�?�(���K�f�b,� R�%nji�"K��fF��Z��MX�!HE���wA����U����@$E<�?/�p����V�"Nk�������O����0���0
����Y1�rw���3��@.54���X�����4Z�wm#.G{&A���j���di���� �) �(58fC�t^/���gE_:�9j�`N�sW����HN���ck��@�}j
|
D�om����������� ���i�>�7([|�K�[�r�����e��'8�'�O���P?��e���=1��UA-�S��N0��*����������\��v������8�SBj�#�=�A��D��nW�e�&��P�����.�q��r��"���K>�
F���3x�N��HB?bj)C��Q�&���W��mB���GNsU4�sQ7]?u�����L'S:�[�����j��9u����n)��4��v��/��V�
i��������y����f��jz�4��W���z UA+[�O�ET�{��
-�o���qoESb3c����}�w������.��+�f�,�D����4�&�]Q�m�[t��>�?����������f�=��H��K=��E����P��I���g����d5���3�H�3�����=��$�I�-�^�;0����z�{��;�������.��(����fG �e�����>������������r��e�~j-V�
P:\x�;�UJEdr�m����/��$nJF�J�#���;TP#��%�P����(�r��c�F�t�j7[��|�R��~�f?�� ��S�5������Z�m��W�r_:�F��1���J�'��J��D�U��
;����� �vP�R�JI��U":2%tA[C���������
�0����4`�� �{R�oA� �m\WaV�\d��[��;6�t� ����E@^��b�%}tv���������|�N������?J�����J�p'��r.��n\]�Im��V7MO� �W���"�'>�$J������})E�w�1~jpr8� ���YPLi����A�oJ)H PZ��*���b��$5{��QO.�t:�-���A�]E�1�H/��v4j��J_�
Gi��H��W�=O�j��<D��-�/@���=�+���%(�L�U��f�Z�A����L����[D����6e�� Q�f+` ��J����
5�(�� KP�(3��-�=�b��w~L���}h7P�A�H'��) �r
���I��:�ue)�j2�m�6���>��F���W��"e���S�;yU�������2O��(I[ ���| ���2/��_��(�#A����I�@� �[H���w���=L�#�=����9.�?/�\*6��������������H�����S�wq����W%��������kTq}wq��v�����^�u��%
��4�.=����
=��q����Z:����40p�~��}o4./?�4D�����V.j��W�r��J8�������
�#��U����[���6��,�v�b}F�Ix��[_8���3u������~�<��y�\�E3#�����4�J�X�9��K��-�mp�{p��]�����
gF�L8�'��AK��F5����e�2u�w2N�Fd8 ���-c�
�q-P��~N�p}>�j��{���j��y���S����\��Z�U
�
/]v��|w3��
�e�C��{s�@��X!o+|�y^�mt*�))���Yz�Q��R���|�����������25VeK����K�$����V������5���d�J=��J�kV%��
(|��{�ZM�Hb����fU�/r�N���LB$��$���B\�R���5d}Y�%�i��3�p��t��\`�ab��x<
�3x �a
�t��M^���G��4!,��n��M�c�R����<}��-��������t��h���jj�(���Lf0�^]��R$J��A�"|(M��./�A!������p�� �F���[��dJ����d�H�h�}��Ey��c"���*����.�������9y�$wsG��+�f%�QE������HNGHW�1
Wx�\�P����)�s��E��nN%�P��t.7�tg���>0���p48/�X�{!�["��f4��-+����9��E*9�VJ#����lG��'����PU�g#��t��;R��~�S�e��|��<4>�g��db��u8��O��u�#��������>/���)bb`��gyR&�N�[i4��u������D�+L}y�%�������&Wq���w���9D�D�B)�����$Id<�msm��+�,Aeg�5=�r x@/���51|�{�R7����M}TD�Y�c�eI��%;���~��'�+�Y�[� Qt���7��������+X���r��e�B����`B�.f��?����-����/�U�!3z�"��>&��G�"���������[(�J�%4��5v�bFgo��[��V������-T[Q�
������������|���4h���1���R�O,�i�����jdx �H,%��80�eO��Gq7��p�s����"�d4��q����0Ol��bd����=p}�0ZPE!)�V+;SJ�O��<���T6D T|� =+��������}(�,l�� ��6cP�S\�+�����=L g��;6��A�u>�x�
��6�|�32B�w����h�*1�����P�y��}����Y�����u0��m
����aiy�Mr�k�������*���GMCl�{�m 6��Y����Z���u����6���� s�5���H�A�?"|Cc�����W�{!�]J�[*��6�(�"[���l��#��%��������������.*
�=h[�{�
�i;�4 �����H8P�M]��j�R�%����P�0GNt���R��#0c>��2��Q� �YWT$�P,�BBF>,�lz)�*����{Q
���W8�9�v�����
rL���Uh��D��pIm�\v���t��=u�GN1��QX
�X,��S���@��z k<��v%U�D�O������<d3%^����x=�;�O������=�P��a{������)K0�^�(�
3U��!�t�����
�,��h�����bZ���>��w���t<��CD7�XB>1�������p���`gJ���q��58��s��M������-��?��8�t�7�F���`P~rVu��� ���.�^�����^�|{qz�=������gf��7��d#���������W����Q��r�XC��@�����iT�����uh��w8�U9���D91�Z��J�C)�%�=��q�������3���OR�P;L�4#w�b;��t���`�OT� ���������c�N|���.������@�����YoE�
k���.�+��J���}T�D�`���/r��X��m��n�,3:�6��*PzA�Q�L��V��cga���19B��Est�����h!�����)�1[*�����z���R������7{f��]��"C7v�+o�p�A���dkL�[Q�u�d2
�������B��~R�W��B��h@L"�~%7N{;7����@L��O
6���s��U��Q�+8A�����r���O���sLh�a=@����������J��}�o US�^R����y�}��=1� �4�T�kR� U���Sp�c�{��?��F�a=��y��0������g��+����A�3N6�������(^�xC
m7Q���@���^��GX�yp�q��LU�5�����+ ����Z���2����x�B:�;���~�F�~P9h�%���}�[d8�i�P5I���$���P65��I��%���=v��SU���*'�g�� �������j��d3�lAH���%
�g�l��
_D��\N��H����l�u� ��r�_S �:�����u�����9+V�l��r��t9�r �(�U;�����r���~���QV�3N�{��-��Fr����{��*�O�2��E���r�pb�
9�XQ�B�$2o����U��}o�C�U��=9���_��}���[L��L�M=�W�7��~��0
|��`�j1�]��v�x=���9���J��('^ han��V��Q~SA�MvC6��R���)=R"�2C�T�*��8�lk��~q�b'�K�n����������15�������Y8h��\74��FB����3n�r7_.�nn���8�r�_�� ����!�O���'JU�2�|���j]a�H�������S
D���IZ��
��-�����#sV�r�i��.�ufo{3�����_#�u�4G�J6�A���B�m�����6fXAEJ/��Kk!y���-�BN�ad�Sz4������6zu6�� W5��$'�S��{�Ci�$Hp��!J#��������@�B��4��BZk2f�l,�����o?���;~���K��G{��m���#:Y���pA�KR��9��hHs�H�U)��;���������?]}�p����K��-��]�e�h6:��h����+�����J���h{���Z�dK�E�L��:h[>P���K�����$vt��a�,�&�W�m�k����]K/8�����(rI���6&jD|t�KTC�����bw9�q�)�3�8�L�����
ru �zO)I��r�4�P����?;R�y�wS����Y0*���������.AAl��d����)C��eb�Y��6q�-ywz�����;��k��]c�
,Y�h�3B��8����0o���d�{�wp{����Ky�H���I���Xy�U�lr�� ����F���o���F;��������f��t*�'������Knf%\Yc�t�����bD�z��#I�5LCy��9���p9/[y�q�����l�F<}��~�>����gk��������-�� T����h��e:�p�f�x�h�W
�s�����J��lh&zb����']�Le�
$����>�>A��`���N���}�Z'������#Y
9j�� i�O������9a*������>�i"�����&�������$xx���d-7A���R����b�"�'&,el�`|����!�������)l��d<�p���F����r��v��M�I��p�Q���[��,���7�k�i���YB8���L��)�p����{�����%{�1k��@w]M�tj0����pUEt?��$+��K�*y��]�bD�\RR^F���-���G�.'���e�a�oO���p(�(N�r���s-m��4ly)�����:����IX��xx ��oy��Q6�[���G�8a���#������9����E�g����t����m�i�cP������U�/������#��w[���*�1\I�?�����i��snMfU���cvm����r����|e=2T�+
������n�K�s�G��d?N�A�.��y�����y�:'������l�����]��'(n(�=j.��'�;��\v6�/�RzE��K�������BW1|Qcp7)�t<y]M�z 'C�� P�$pa|+`���[pP!��|��G��W�m�`�iJ�e��u]� nte�D��v����,6�����f>A����w]���o��S�Ao0�����m�1B�8}c��:^S~��j�b�eD�����
*���y-�g��7&�����%g�-�B�tB$��H���i��ws������Aj��?[��b��b�%��h������o���-hC��{�^aiPf#!RJ6D���\�*C���j�������IP1g�[F'��2�`�(�TSU�Z]� �;�E�&[���x�t��k��U�����J��8f��4��
63h��j&gG{���95dt�7��p�n�u���C��~�
�F���D��z���
*%�TX����Gl=�'��H����D8�k�D�\��)��t*C����7^�������z|��:����{j��
����+';�H�iO�|�E�]�����&��%Z%��aQ8�:��v��C �7��u;m:�Db���{k��E'�^G���+3��n50I:
��*��K���,�@*;(&��HJn������I��\��R�[MZ�E���j�@���jv��d��l6*r2[[�M r���
����;=2����7��;j�YW�"X�qu�auJ����gS4���!�����D6F���(��2�e�fX���BX��[��Y������Vu�����
sh�Y�>�oO���u+�O*���h��~�����P����r�8vGf@V ���u��T�R_��ON������D�f�A��������t �r;���o��5a�)HV@���t�����L�(��U1�.+�������{ e���S�4[�I���
�D�4*(�������(^��/dWb������I�����
���Nz�PaT�*!D�pUv���%E9�J�$��`�-��!�?�Yc_��E�Tm��Q0�`v*� �H��aF� ���e?���%3�:���O3�e8��r�e��1����]%��tFGE�8����%N�GG@��8;,(rny�\� 7+zyR�j*',� g����1��
�0����9�%Ui�};i��������q��:6�p�-I!�����|�(�-L/K�b�sTh��_��2y�?��Wo������kn�����U�P�E�,D`CRxQ�E�!�r���[ ]_���U��6��Uvt ������Z�@�0�v[N6��QVD�<W��'���A�B`v���D� ���m</��PYV����RDHKZ v���P�hg�����,�!ol���8�Fg/O��K�������������Fy�`T��=�������/���4Q�%�8�~���;V���y����W_Y����@�&7"r� �����*���������X��S�)��
Q J�9���]�2 b*A��
eS������qn
C�����[��`A����#�P=���i����s�,�lo9h�9CI
���@G�fD�����atF��
���a���k���||5����������!�<R��K����!��8�c�YYs{-)hE ���p��r�nj���EJ���-�ra�6p�BbF�=��KX��Z>�iO�P�{F�R:'q3s�_n�f�s_&����V�4�(mS�� ��0I5���A6�����> �p�aQ(`R�v���y���3~fj�:������N[�#(Cq0u����|������)����S����&���c����C/��-��rM�
�2��~����!y���%w���+:Zq�R@XBp}f t��I�i��G��T�I���Cy~@o���� s2X'
Y5u`����X�3�_ �!�������7��I�uU�w�� ��X�O0�t�'����1_d�r��;�_�fN�{���Jtl}�r�b�N���NZ~"K�71���YDCDs��H_��
����)vnz���,H�]2d���{��Q����0�q����YX�������}�yc�N�,��Hn.���vO\��Oa (�%���K"�V3�i81�k���J����j�y�wB'�
���&����
�p�4�[����&��$��W���f�b����JVA���+5q�h�19�V��J�N�4�"6����C�����GGV�l�6��=�%#���y:����k����l�����{ewn}�����'������5�X���ySs��+[)|����3�c���9\�����P�����h�)>�x=�2��o��L�!�<����M�����0�-�w���nH���|V����-���D��f�$9�3�m�]w����x��GW�fb�����0�:�v6DQ}U�I���*������}����y���x�����%���S(�}}y�T�+ETr�-�r��p��/gp�h�T�y�e���c�Wm�/x�OF����dv���^&��C��������k��e�,%�"7k��E���������z�����w��b�##�y��?����,�C�J�u���Z��,��-�{l���������:",x��P�i�M�^��<l�j��f���D�^�����ju�������#���2�7�P ���x���v���=v�����M�y�<�_�%/�`�M�)��/�A���z�z��8e6&c;`:*����F��0F��;�����7���������&!X98����r����k�$.����[����t�U
���L����c����dS��-�����p���W��1^fqo��szv����wQB��S���N�V%����G��P\��j�EFz����I�NO�=�� ,;TH��&o�N���
D��v��eWo��(Y%Pq$H��Z���:P����[��@{SQ�m�����\m��*�|������lhV��
@��H��@�1$8W�Q���|O����k��O�J�n��Q�g��S����e��d�TY(�T�#�t���np�"#�:G`�b�|O��P������`���kS��vQh1�F\RU����l�Q$�o�)��}[�5?\�uz���1��* j>�q�i�0\���P�+��1|P���=#�y��7�dd����K���r��o����n6L��\�:����u���<���*�p� R�>W�;����9U�rS�������|7����Y��9~<GW����,1:QG���4�Z@rMh�hK$+Wg�X��/�8���<z=����=�EA#���������W�
P��o��$w���#U�Gz�e����J��
2'/��*c����_N���{�����5p����n����igc��$T���qp��g�a��N����9[y�T��*m���x�-��V�����,�<��-O��"�a�O��B�{rI'u�?�>����%�$+�k]��"�N���Cq��O����e��-eZ[��DwC>�\��\�"�[1_�{�Ym�k��d�Dr�U�T �[��Tk/`���lU3��A��| ;��O�� ��������������NS�.��r�>�b] ��k�fV3w�����mM��R��Mv��N�/������lb�D&Z�a%�FD�a��H�����>���6��2���U�~�4I�28Z'��\D��'�m�;�I*��b9�����6�fhk|[T�x�SL���M�����R�+_���\���D�~�RHT[@%�x��[�5�m��(C�Q�><lU���g��8f�
E��g+�q���X�O����[�;er��#Zb�o�1~��6��dQ�K�.�k��Y�����������If�����b�������K����n�\u�����.��?����YD�1�������>�O�����:�g:�o�[� �Ac^������v����0M�+����K�-��h8��:z���H��{(U%4%�a=�gD�:�Y�2�l
V)6j�Q�.�ngX�s9w=3� FV���sW��qx�^����T�������2b�/@�+��g�����g���V�n���������^��z������*�e����2>h8K5��a&S���w�X0|����Z����B��O�������N}�����V_N����7�� �+�
(�"���������l�pO%rE�)�����*���
�H,�n����E����I��V&�o-oM"J3N Z�����2��@�e�<, ��\���'�p($N��fA�8���(���9D��0/�U�#�Bm�))��>�4�|d���7������k������e�3�[&��O[`�*l��t�a\\���'�z�2��?����}��~0��@�d����ku��8>��s������Y���e���� �^D��u��z ��� �u����1xH�(6��t�
��a0��h���g�����|�!�TG,����.�*��� ���$�e&i:J��%6�G���D��r���r��I �v�<S�CEWtf��i:c����I�r����|dQ�����0s�����:8�@���O���:N�
�<1���r3�M�x��0I��=^�������>���F��P��z���}O���3L���� �%���0^��w C��jl��K.� w ��-&�Z�����b�,��gzi�SV%�������������X8�-�\ �y���f��~��y�����g�!���?�D�������9-��u,��n�
��=�������;���vk�y���h���4��F����f��������f,��U����Y^��>�"_���7������?K��]�������_HT
��������w�_
�K��J|s��#����������/�_�^;�;#"��?��d����C6Af}��C�����E�;}�3���=&�!t�]��+��R�r�>�7�aP���d����]���d�Od#� Z�{�~U�S!�0��3� ��W �a��>�N�����'���\����n�C���S�d�� Z����H6�)R R���-�-
}s`Jx�����������:Nl ���;'a�7��p��
3��/�[`����Kxd�6Q�������O1�7}5 ��\N�����y������f�������QP��7d��O�_"#�I�� G~+�
��u(>��A�I�I��a �D=���>��B��}�c�=hT��G�s��$�:��" �]��|�4h��V��'C�2���dC�D_z�k���5��+0Dk��<����/bE*c�@Uu
%fE�c6ED�N����JMq:s�����OZ��C�"��C-@r�{/
HTf^]N�����X3����H��~B�<_��h���fz�M�%e ���_E3)�w4A�Tb��7�/��b�@j�p|��u�����9���M1��/������J�����"�R@e�U/��������H �
(�d{M��@F����G�
g���QE�_pB\XV����xF��J�:�0��W�N6X���eW5b�;-����X��%7"���)�X?���#�f�0�gn�������>������:q�����&�x��"#�YW9c�`���/�$5����4 OEe+����MB�PQ
�[NHD�mz�i8(S~
&�3@2cAL��d:�Q�9k ���R���(�U�+�+�����p��K �Er�"p>e�B�g���/@��k�
%�����k�3��5%�������Y�~*��Z��W!He|O"�T�0~��J������m(��Y�*}&0;��@�7������
��/9�l~YJL��W6��#�CxS
�+���
�p�0�5E&������'�2RX��IM��Z5����k��T��}�"c��@�C?�'�&������b��pu|Wz��'&�G�xaV�nt��k��~�����
�&�����(ZN]0�������Y@ d4fGb���H`��O�/����I9�wq�;��=S�'�4R)��PW���g�.S���)��U1Nh�l������@Krr���m#���8����y��1�q��m02������ ? � ��TX�"W��&\��6��)���
�����Y=5s�!�Sw���6�Q���������o��% d���m�e���8�
!\�$vSS��Iz����7��Hwrf�8���%o_W7��!N�hm-
���JKc���f����n�V����`m-���J-����Mu�*��$�xx��y2{��/#Q����,�Oo��H
�Q�C����:e�2��? F���A�D��Q�'��a��9�:
>�k�]k2O,���H�����}�*��{U5@�H1�T$������h'�uZWl��f����I���`�������U�Y�zn"(}q������l��<dZ}��Rj,�a*����[��R��%_9I@m��t��5���:�������"'���s`�����q�8�k��Gn���:V��8iC0��>����G0�>��GL������r���-��B4�
���k�e��R4���{Nu�}_��He,Df�xYfcQT�`6qH�\=# %���s�n t"B$<�z{�n�u��ZD�V�'�LE6U*"K�t)M��������c�l����
��i���ic�*d���>�@��o��YK>N�!ka��v�����t���J�I�QT�����H�����b���h�������?���R�?�h7�����y<u_�>����s�o�X>�c���� C"([��JX�3��L�s@y�i�m�����2�P����d���)�,x��DA^������G��.�2�9l�*�����l`dC#�Xu�8�����
!�4�nmUZ�
��sn
��D`7��7F���Ji���}��e|�@��B��
;����th�g��X�?�Z��t� U���u��B ���.�y�:���R�h��X����>�I��
f�����9rG8�7@�`H��>���U�UT����S���-'�Cn���%��P��K7���|�a��q�!�
������������q�������vk�����������w/�[��pg�|��?��[
g���{a.L�FN�-2�F� �c�Y����(��XU,�g���MWp �����H�n�H����l�M��/��+vRLX�(�A^@!u��w���2K�a�
��^��"�����v&a���wl������������uK�
�U����e{-i.E��4<i:J�y����KBDi�t�F>U�A4o�
�x�5�D'���� C��@ � ��P�`!�N�[in|g��h��|��
"��i��r���_��Hn�;V>��`mq��<�ndV�/0���H��&���d7Pq�1��j���k�*�A������ $�RN'R����M���pw(���C{G/���M ��l��L��>�B<��������X���[������"x���?�����|�=P�KL����*��. ����-3�~���W�Rh��h�G����m�6�q��&t���\J%}�#�:f�+�-�`1�����_�u����e�H�Uh�H��:��i61�5�����) ���u_9�R�o���k���}G�����^��F���l�:G�E�A#�)yh���&"������l�fk TV�a$��l�+�b���&}�Vc�-C��gzH.8�Hn���%
F'b����=��x�4��M��3p�0���������V��k�0H��V7 �?_�v�t�����N�v��~�p������l%����~������4k�/��b]'�.���g����
����&�5��,"��^����� 0�H��x�
� �!9[���9}�YYx`�P��+oLdq1]�3sX�Q8�F��7��#;����
Or4����J� Q�:&��?Y���������&�Or��5Y?s�]i������%��5�<Jh
��BlDI��E���f�r�iv8��f�������G�u[��&a��j���?����� u����^s�\��^T^������O�l��WZ�l�_�k[�`�����%��z`X/]h�S��qDv�+��8Cydo��b�M���A�WP\F�S/^��7}!2W��Es��\^�S/��k�t�I���By~4���q�V�vZ��a+��g��&�Y��qj �?[�)��c��[/���|X�=)���N�-�a���R$\��������8�����O.�f���4-&�T�t�{�����\��r{�������� 1�+:<��_��9����������cnl��.z���]������o����$����up�z�&r�et�S�����v(�Kp��@��w��sv{���� e$��gX�����h�`��mH�=#B�K��p�]�{���borQ�)2��� ��q�j������� ��4)�nGMu����y��dP������}�N��X.��'sy�J�1���7��/������RG�E)#���{���<��=+r����R�(��89�W����R��`����?1�U��j������r�p�@�A�C#B���wQ$���;����v�h�w���"�Z�����A�����! �ARM��: a6�� ��Z�D�!FY���>f�8����]�G��4��� �����)d�z����">|w��()�e��"+�>�u������n�, �Y�KO.!�X�������:��B��hxc�V�
�r+�W�}� �M��@�(��i�*�Ua�hET_�:��;��"�/�Q
fem�d@� �:�O����!$(��h����x
9�h)��t�&'���k7��-�c�?l
�$<��vF�rQ�������Eww�&Y6���!��H����`9�;#�}[���"9o�6�/��JJ���%��>�I��j���:|��<���B��.� g.�KEt�; (���W��{+v��7^����T��4�*��Z��rF1�Hq��V!�����Z~T�\l-e�&C�h�T�Ci!@������k
�I�f��Q�&I�����"