Use partition column's collation for range distributed tables
Don't allow non deterministic collations for hash distributed tables
CoPartitionedTables: don't compare unequal types
DESCRIPTION: Expression in reference join
Fixed: #2582
This patch allows arbitrary expressions in the join clause when joining to a reference table. An example of such joins could be found in CHbenCHmark queries 7, 8, 9 and 11; `mod((s_w_id * s_i_id),10000) = su_suppkey` and `ascii(substr(c_state,1,1)) = n2.n_nationkey`. Since the join is on a reference table these queries are able to be pushed down to the workers.
To implement these queries we will widen the `IsJoinClause` predicate to not check if the expressions are a type `Var` after stripping the implicit coerciens. Instead we define a join clause when the `Var`'s in a clause come from more than 1 table.
This allows more clauses to pass into the logical planner's `MultiNodeTree(...)` planning function. To compensate for this we tighten down the `LocalJoin`, `SinglePartitionJoin` and `DualPartitionJoin` to check for direct column references when planning. This allows the planner to work with arbitrary join expressions on reference tables.
Postgres doesn't require you to add all columns that are in the target list to
the GROUP BY when you group by a unique column (or columns). It even actively
removes these group by clauses when you do.
This is normally fine, but for repartition joins it is not. The reason for this
is that the temporary tables don't have these primary key columns. So when the
worker executes the query it will complain that it is missing columns in the
group by.
This PR fixes that by adding an ANY_VALUE aggregate around each variable in
the target list that does is not contained in the group by or in an aggregate.
This is done only for repartition joins.
The ANY_VALUE aggregate chooses the value from an undefined row in the
group.
We've changed the logic for pulling RTE_RELATIONs in #3109 and
non-colocated subquery joins and partitioned tables.
@onurctirtir found this steps where I traced back and found the issues.
While looking into it in more detail, we decided to expand the list in a
way that the callers get all the relevant RTE_RELATIONs RELKIND_RELATION,
RELKIND_PARTITIONED_TABLE, RELKIND_FOREIGN_TABLE and RELKIND_MATVIEW.
These are all relation kinds that Citus planner is aware of.
This causes no behaviorial changes, only organizes better to implement modifying CTEs
Also rename ExtactInsertRangeTableEntry to ExtractResultRelationRTE,
as the source of this function didn't match the documentation
Remove Task's upsertQuery in favor of ROW_MODIFY_NONCOMMUTATIVE
Split up AcquireExecutorShardLock into more internal functions
Tests: Normalize multi_reference_table multi_create_table_constraints
If a query is router executable, it hits a single shard and therefore has a
single task associated with it. Therefore there is no need to sort the task list
that has a single element.
Also we already have a list of active shard placements, sending it in param
and reuse it.
Before this commit, shardPlacements were identified with shardId, nodeName
and nodeport. Instead of using nodeName and nodePort, we now use nodeId
since it apparently has performance benefits in several places in the
code.
Before this commit, round-robin task assignment policy was relying
on the taskId. Thus, even inside a transaction, the tasks were
assigned to different nodes. This was especially problematic
while reading from reference tables within transaction blocks.
Because, we had to expand the distributed transaction to many
nodes that are not necessarily already in the distributed transaction.
In this context, we define "Fast Path Planning for SELECT" as trivial
queries where Citus can skip relying on the standard_planner() and
handle all the planning.
For router planner, standard_planner() is mostly important to generate
the necessary restriction information. Later, the restriction information
generated by the standard_planner is used to decide whether all the shards
that a distributed query touches reside on a single worker node. However,
standard_planner() does a lot of extra things such as cost estimation and
execution path generations which are completely unnecessary in the context
of distributed planning.
There are certain types of queries where Citus could skip relying on
standard_planner() to generate the restriction information. For queries
in the following format, Citus does not need any information that the
standard_planner() generates:
SELECT ... FROM single_table WHERE distribution_key = X; or
DELETE FROM single_table WHERE distribution_key = X; or
UPDATE single_table SET value_1 = value_2 + 1 WHERE distribution_key = X;
Note that the queries might not be as simple as the above such that
GROUP BY, WINDOW FUNCIONS, ORDER BY or HAVING etc. are all acceptable. The
only rule is that the query is on a single distributed (or reference) table
and there is a "distribution_key = X;" in the WHERE clause. With that, we
could use to decide the shard that a distributed query touches reside on
a worker node.
We update column attributes of various clauses for a query
inluding target columns, select clauses when we introduce
new range table entries in the query.
It seems having clause column attributes were not updated.
This fix resolves the issue
Description: Support round-robin `task_assignment_policy` for queries to reference tables.
This PR allows users to query multiple placements of shards in a round robin fashion. When `citus.task_assignment_policy` is set to `'round-robin'` the planner will use a round robin scheduling feature when multiple shard placements are available.
The primary use-case is spreading the load of reference table queries to all the nodes in the cluster instead of hammering only the first placement of the reference table. Since reference tables share the same path for selecting the shards with single shard queries that have multiple placements (`citus.shard_replication_factor > 1`) this setting also allows users to spread the query load on these shards.
For modifying queries we do not apply a round-robin strategy. This would be negated by an extra reordering step in the executor for such queries where a `first-replica` strategy is enforced.
* Change worker_hash_partition_table() such that the
divergence between Citus planner's hashing and
worker_hash_partition_table() becomes the same.
* Rename single partitioning to single range partitioning.
* Add single hash repartitioning. Basically, logical planner
treats single hash and range partitioning almost equally.
Physical planner, on the other hand, treats single hash and
dual hash repartitioning almost equally (except for JoinPruning).
* Add a new GUC to enable this feature
- changes in ruleutils_11.c is reflected
- vacuum statement api change is handled. We now allow
multi-table vacuum commands.
- some other function header changes are reflected
- api conflicts between PG11 and earlier versions
are handled by adding shims in version_compat.h
- various regression tests are fixed due output and
functionality in PG1
- no change is made to support new features in PG11
they need to be handled by new commit
After this change all the logic related to shard data fetch logic
will be removed. Planner won't plan any ShardFetchTask anymore.
Shard fetch related steps in real time executor and task-tracker
executor have been removed.
This is the first of series of window function work.
We can now support window functions that can be pushed down to workers.
Window function must have distribution column in the partition clause
to be pushed down.
With this commit, Citus recursively plans subqueries that
are not safe to pushdown, in other words, requires a merge
step.
The algorithm is simple: Recursively traverse the query from bottom
up (i.e., bottom meaning the leaf queries). On each level, check
whether the query is safe to pushdown (or a single repartition
subquery). If the answer is yes, do not touch that subquery. If the
answer is no, plan the subquery seperately (i.e., create a subPlan
for it) and replace the subquery with a call to
`read_intermediate_results(planId, subPlanId)`. During the the
execution, run the subPlans first, and make them avaliable to the
next query executions.
Some of the queries hat this change allows us:
* Subqueries with LIMIT
* Subqueries with GROUP BY/DISTINCT on non-partition keys
* Subqueries involving re-partition joins, router queries
* Mixed usage of subqueries and CTEs (i.e., use CTEs in
subqueries as well). Nested subqueries as long as we
support the subquery inside the nested subquery.
* Subqueries with local tables (i.e., those subqueries
has the limitation that they have to be leaf subqueries)
* VIEWs on the distributed tables just works (i.e., the
limitations mentioned below still applies to views)
Some of the queries that is still NOT supported:
* Corrolated subqueries that are not safe to pushdown
* Window function on non-partition keys
* Recursively planned subqueries or CTEs on the outer
side of an outer join
* Only recursively planned subqueries and CTEs in the FROM
(i.e., not any distributed tables in the FROM) and subqueries
in WHERE clause
* Subquery joins that are not on the partition columns (i.e., each
subquery is individually joined on partition keys but not the upper
level subquery.)
* Any limitation that logical planner applies such as aggregate
distincts (except for count) when GROUP BY is on non-partition key,
or array_agg with ORDER BY
Postgres provides OS agnosting formatting macros for
formatting 64 bit numbers. Replaced %ld %lu with
INT64_FORMAT and UINT64_FORMAT respectively.
Also found some incorrect usages of formatting
flags and fixed them.
This GUC has two settings, 'always' and 'never'. When it's set to
'never' all behavior stays exactly as it was prior to this commit. When
it's set to 'always' only SELECT queries are allowed to run, and only
secondary nodes are used when processing those queries.
Add some helper functions:
- WorkerNodeIsSecondary(), checks the noderole of the worker node
- WorkerNodeIsReadable(), returns whether we're currently allowed to
read from this node
- ActiveReadableNodeList(), some functions (namely, the ones on the
SELECT path) don't require working with Primary Nodes. They should call
this function instead of ActivePrimaryNodeList(), because the latter
will error out in contexts where we're not allowed to write to nodes.
- ActiveReadableNodeCount(), like the above, replaces
ActivePrimaryNodeCount().
- EnsureModificationsCanRun(), error out if we're not currently allowed
to run queries which modify data. (Either we're in read-only mode or
use_secondary_nodes is set)
Some parts of the code were switched over to use readable nodes instead
of primary nodes:
- Deadlock detection
- DistributedTableSize,
- the router, real-time, and task tracker executors
- ShardPlacement resolution
- master_add_node enforces that there is only one primary per group
- there's also a trigger on pg_dist_node to prevent multiple primaries
per group
- functions in metadata cache only return primary nodes
- Rename ActiveWorkerNodeList -> ActivePrimaryNodeList
- Rename WorkerGetLive{Node->Group}Count()
- Refactor WorkerGetRandomCandidateNode
- master_remove_node only complains about active shard placements if the
node being removed is a primary.
- master_remove_node only deletes all reference table placements in the
group if the node being removed is the primary.
- Rename {Node->NodeGroup}HasShardPlacements, this reflects the behavior it
already had.
- Rename DeleteAllReferenceTablePlacementsFrom{Node->NodeGroup}. This also
reflects the behavior it already had, but the new signature forces the
caller to pass in a groupId
- Rename {WorkerGetLiveGroup->ActivePrimaryNode}Count
Comes with a few changes:
- Change the signature of some functions to accept groupid
- InsertShardPlacementRow
- DeleteShardPlacementRow
- UpdateShardPlacementState
- NodeHasActiveShardPlacements returns true if the group the node is a
part of has any active shard placements
- TupleToShardPlacement now returns ShardPlacements which have NULL
nodeName and nodePort.
- Populate (nodeName, nodePort) when creating ShardPlacements
- Disallow removing a node if it contains any shard placements
- DeleteAllReferenceTablePlacementsFromNode matches based on group. This
doesn't change behavior for now (while there is only one node per
group), but means in the future callers should be careful about
calling it on a secondary node, it'll delete placements on the primary.
- Create concept of a GroupShardPlacement, which represents an actual
tuple in pg_dist_placement and is distinct from a ShardPlacement,
which has been resolved to a specific node. In the future
ShardPlacement should be renamed to NodeShardPlacement.
- Create some triggers which allow existing code to continue to insert
into and update pg_dist_shard_placement as if it still existed.
* Enabling physical planner for subquery pushdown changes
This commit applies the logic that exists in INSERT .. SELECT
planning to the subquery pushdown changes.
The main algorithm is followed as :
- pick an anchor relation (i.e., target relation)
- per each target shard interval
- add the target shard interval's shard range
as a restriction to the relations (if all relations
joined on the partition keys)
- Check whether the query is router plannable per
target shard interval.
- If router plannable, create a task
* Add union support within the JOINS
This commit adds support for UNION/UNION ALL subqueries that are
in the following form:
.... (Q1 UNION Q2 UNION ...) as union_query JOIN (QN) ...
In other words, we currently do NOT support the queries that are
in the following form where union query is not JOINed with
other relations/subqueries :
.... (Q1 UNION Q2 UNION ...) as union_query ....
* Subquery pushdown planner uses original query
With this commit, we change the input to the logical planner for
subquery pushdown. Before this commit, the planner was relying
on the query tree that is transformed by the postgresql planner.
After this commit, the planner uses the original query. The main
motivation behind this change is the simplify deparsing of
subqueries.
* Enable top level subquery join queries
This work enables
- Top level subquery joins
- Joins between subqueries and relations
- Joins involving more than 2 range table entries
A new regression test file is added to reflect enabled test cases
* Add top level union support
This commit adds support for UNION/UNION ALL subqueries that are
in the following form:
.... (Q1 UNION Q2 UNION ...) as union_query ....
In other words, Citus supports allow top level
unions being wrapped into aggregations queries
and/or simple projection queries that only selects
some fields from the lower level queries.
* Disallow subqueries without a relation in the range table list for subquery pushdown
This commit disallows subqueries without relation in the range table
list. This commit is only applied for subquery pushdown. In other words,
we do not add this limitation for single table re-partition subqueries.
The reasoning behind this limitation is that if we allow pushing down
such queries, the result would include (shardCount * expectedResults)
where in a non distributed world the result would be (expectedResult)
only.
* Disallow subqueries without a relation in the range table list for INSERT .. SELECT
This commit disallows subqueries without relation in the range table
list. This commit is only applied for INSERT.. SELECT queries.
The reasoning behind this limitation is that if we allow pushing down
such queries, the result would include (shardCount * expectedResults)
where in a non distributed world the result would be (expectedResult)
only.
* Change behaviour of subquery pushdown flag (#1315)
This commit changes the behaviour of the citus.subquery_pushdown flag.
Before this commit, the flag is used to enable subquery pushdown logic. But,
with this commit, that behaviour is enabled by default. In other words, the
flag is now useless. We prefer to keep the flag since we don't want to break
the backward compatibility. Also, we may consider using that flag for other
purposes in the next commits.
* Require subquery_pushdown when limit is used in subquery
Using limit in subqueries may cause returning incorrect
results. Therefore we allow limits in subqueries only
if user explicitly set subquery_pushdown flag.
* Evaluate expressions on the LIMIT clause (#1333)
Subquery pushdown uses orignal query, the LIMIT and OFFSET clauses
are not evaluated. However, logical optimizer expects these expressions
are already evaluated by the standard planner. This commit manually
evaluates the functions on the logical planner for subquery pushdown.
* Better format subquery regression tests (#1340)
* Style fix for subquery pushdown regression tests
With this commit we intented a more consistent style for the
regression tests we've added in the
- multi_subquery_union.sql
- multi_subquery_complex_queries.sql
- multi_subquery_behavioral_analytics.sql
* Enable the tests that are temporarily commented
This commit enables some of the regression tests that were commented
out until all the development is done.
* Fix merge conflicts (#1347)
- Update regression tests to meet the changes in the regression
test output.
- Replace Ifs with Asserts given that the check is already done
- Update shard pruning outputs
* Add view regression tests for increased subquery coverage (#1348)
- joins between views and tables
- joins between views
- union/union all queries involving views
- views with limit
- explain queries with view
* Improve btree operators for the subquery tests
This commit adds the missing comprasion for subquery composite key
btree comparator.
So far citus used postgres' predicate proofing logic for shard
pruning, except for INSERT and COPY which were already optimized for
speed. That turns out to be too slow:
* Shard pruning for SELECTs is currently O(#shards), because
PruneShardList calls predicate_refuted_by() for every
shard. Obviously using an O(N) type algorithm for general pruning
isn't good.
* predicate_refuted_by() is quite expensive on its own right. That's
primarily because it's optimized for doing a single refutation
proof, rather than performing the same proof over and over.
* predicate_refuted_by() does not keep persistent state (see 2.) for
function calls, which means that a lot of syscache lookups will be
performed. That's particularly bad if the partitioning key is a
composite key, because without a persistent FunctionCallInfo
record_cmp() has to repeatedly look-up the type definition of the
composite key. That's quite expensive.
Thus replace this with custom-code that works in two phases:
1) Search restrictions for constraints that can be pruned upon
2) Use those restrictions to search for matching shards in the most
efficient manner available:
a) Binary search / Hash Lookup in case of hash partitioned tables
b) Binary search for equal clauses in case of range or append
tables without overlapping shards.
c) Binary search for inequality clauses, searching for both lower
and upper boundaries, again in case of range or append
tables without overlapping shards.
d) exhaustive search testing each ShardInterval
My measurements suggest that we are considerably, often orders of
magnitude, faster than the previous solution, even if we have to fall
back to exhaustive pruning.
Soon shard pruning will be optimized not to generally work linearly
anymore. Thus we can't print the pruned shard intervals as currently
done anymore.
The current printing of shard ids also prevents us from running tests
in parallel, as otherwise shard ids aren't linearly numbered.
With this change we add an option to add a node without replicating all reference
tables to that node. If a node is added with this option, we mark the node as
inactive and no queries will sent to that node.
We also added two new UDFs;
- master_activate_node(host, port):
- marks node as active and replicates all reference tables to that node
- master_add_inactive_node(host, port):
- only adds node to pg_dist_node
Custom Scan is a node in the planned statement which helps external providers
to abstract data scan not just for foreign data wrappers but also for regular
relations so you can benefit your version of caching or hardware optimizations.
This sounds like only an abstraction on the data scan layer, but we can use it
as an abstraction for our distributed queries. The only thing we need to do is
to find distributable parts of the query, plan for them and replace them with
a Citus Custom Scan. Then, whenever PostgreSQL hits this custom scan node in
its Vulcano style execution, it will call our callback functions which run
distributed plan and provides tuples to the upper node as it scans a regular
relation. This means fewer code changes, fewer bugs and more supported features
for us!
First, in the distributed query planner phase, we create a Custom Scan which
wraps the distributed plan. For real-time and task-tracker executors, we add
this custom plan under the master query plan. For router executor, we directly
pass the custom plan because there is not any master query. Then, we simply let
the PostgreSQL executor run this plan. When it hits the custom scan node, we
call the related executor parts for distributed plan, fill the tuple store in
the custom scan and return results to PostgreSQL executor in Vulcano style,
a tuple per XXX_ExecScan() call.
* Modify planner to utilize Custom Scan node.
* Create different scan methods for different executors.
* Use native PostgreSQL Explain for master part of queries.
With this commit, we implemented some basic features of reference tables.
To start with, a reference table is
* a distributed table whithout a distribution column defined on it
* the distributed table is single sharded
* and the shard is replicated to all nodes
Reference tables follows the same code-path with a single sharded
tables. Thus, broadcast JOINs are applicable to reference tables.
But, since the table is replicated to all nodes, table fetching is
not required any more.
Reference tables support the uniqueness constraints for any column.
Reference tables can be used in INSERT INTO .. SELECT queries with
the following rules:
* If a reference table is in the SELECT part of the query, it is
safe join with another reference table and/or hash partitioned
tables.
* If a reference table is in the INSERT part of the query, all
other participating tables should be reference tables.
Reference tables follow the regular co-location structure. Since
all reference tables are single sharded and replicated to all nodes,
they are always co-located with each other.
Queries involving only reference tables always follows router planner
and executor.
Reference tables can have composite typed columns and there is no need
to create/define the necessary support functions.
All modification queries, master_* UDFs, EXPLAIN, DDLs, TRUNCATE,
sequences, transactions, COPY, schema support works on reference
tables as expected. Plus, all the pre-requisites associated with
distribution columns are dismissed.
We used to disable router planner and executor
when task executor is set to task-tracker.
This change enables router planning and execution
at all times regardless of task execution mode.
We are introducing a hidden flag enable_router_execution
to enable/disable router execution. Its default value is
true. User may disable router planning by setting it to false.
This commit adds INSERT INTO ... SELECT feature for distributed tables.
We implement INSERT INTO ... SELECT by pushing down the SELECT to
each shard. To compute that we use the router planner, by adding
an "uninstantiated" constraint that the partition column be equal to a
certain value. standard_planner() distributes that constraint to all
the tables where it knows how to push the restriction safely. An example
is that the tables that are connected via equi joins.
The router planner then iterates over the target table's shards,
for each we replace the "uninstantiated" restriction, with one that
PruneShardList() handles. Do so by replacing the partitioning qual
parameter added in multi_planner() with the current shard's
actual boundary values. Also, add the current shard's boundary values to the
top level subquery to ensure that even if the partitioning qual is
not distributed to all the tables, we never run the queries on the shards
that don't match with the current shard boundaries. Finally, perform the
normal shard pruning to decide on whether to push the query to the
current shard or not.
We do not support certain SQLs on the subquery, which are described/commented
on ErrorIfInsertSelectQueryNotSupported().
We also added some locking on the router executor. When an INSERT/SELECT command
runs on a distributed table with replication factor >1, we need to ensure that
it sees the same result on each placement of a shard. So we added the ability
such that router executor takes exclusive locks on shards from which the SELECT
in an INSERT/SELECT reads in order to prevent concurrent changes. This is not a
very optimal solution, but it's simple and correct. The
citus.all_modifications_commutative can be used to avoid aggressive locking.
An INSERT/SELECT whose filters are known to exclude any ongoing writes can be
marked as commutative. See RequiresConsistentSnapshot() for the details.
We also moved the decison of whether the multiPlan should be executed on
the router executor or not to the planning phase. This allowed us to
integrate multi task router executor tasks to the router executor smoothly.
This commit completes having support in Citus by adding having support for
real-time and task-tracker executors. Multiple tests are added to regression
tests to cover new supported queries with having support.
So far placements were assigned an Oid, but that was just used to track
insertion order. It also did so incompletely, as it was not preserved
across changes of the shard state. The behaviour around oid wraparound
was also not entirely as intended.
The newly introduced, explicitly assigned, IDs are preserved across
shard-state changes.
The prime goal of this change is not to improve ordering of task
assignment policies, but to make it easier to reference shards. The
newly introduced UpdateShardPlacementState() makes use of that, and so
will the in-progress connection and transaction management changes.
Related to #786
This change adds the `pg_dist_node` table that contains the information
about the workers in the cluster, replacing the previously used
`pg_worker_list.conf` file (or the one specified with `citus.worker_list_file`).
Upon update, `pg_worker_list.conf` file is read and `pg_dist_node` table is
populated with the file's content. After that, `pg_worker_list.conf` file
is renamed to `pg_worker_list.conf.obsolete`
For adding and removing nodes, the change also includes two new UDFs:
`master_add_node` and `master_remove_node`, which require superuser
permissions.
'citus.worker_list_file' guc is kept for update purposes but not used after the
update is finished.
is now a `::regtype` using the qualified name of the column type,
not the column type OID which may differ between master/worker nodes.
Test coverage of a hash reparitition using a UDT as the join column.
Note that the UDFs `worker_hash_partition_table` and `worker_range_partition_table`
are unchanged, and rightly expect an OID for the column type; but the
planner code building the commands now allows for `::regtype` casting
to do its magic.
Fixescitusdata/citus#111.
We remove schema name parameter from worker_fetch_foreign_file and
worker_fetch_regular_table functions. We now send schema name
concatanated with table name.
Fixes#676
We added old versions (i.e. without schema name) of worker_apply_shard_ddl_command,
worker_fetch_foreign_file and worker_fetch_regular_table back. During function call
of one of these functions, we set schema name as public schema and call the newer
version of the functions.
It turns out some tests exercised this behavior, but removing it should
have no ill effects. Besides, both copy and INSERT disallow NULLs in a
table's partition column.
Fixes a bug where anti-joins on hash-partitioned distributed tables
would incorrectly prune shards early, result in incorrect results (test
included).
For CITUS_RTE_RELATION type fragments, reloading shardIntervals from the
database is rather expensive. So store a pointer to the full shard
interval, instead of just the shard id. There's no new memory lifetime
hazards here, because we already passed a pointer to the shardInterval's
->shardId field around.
The plan time for the query in issue #607 goes from 2889 ms to 106 ms.
with this change.
By far the most expensive part of ShardIntervalsOverlap() is computing
the function to use to determine overlap. Luckily we already have that
computed and cached.
The plan time for the query in issue #607 goes from 8764 ms to 2889 ms
with this change.
Single table repartition subqueries now support count(distinct column)
and count(distinct (case when ...)) expressions. Repartition query
extracts column used in aggregate expression and adds them to target
list and group by list, master query stays the same (count (distinct ...))
but attribute numbers inside the aggregate expression is modified to
reflect changes in repartition query.
Fixes#477
This change fixes the compile time warning message in BuildMapMergeJob in
multi_physical_planner.c about mixed declarations and code. Basically, the
problematic declaration is moved up so that no expression is before it.
Some small parts of citus currently require superuser privileges; which
is obviously not desirable for production scenarios. Run these small
parts under superuser privileges (we use the extension owner) to avoid
that.
This does not yet coordinate grants between master and workers. Thus it
allows to create shards, load data, and run queries as a non-superuser,
but it is not easily possible to allow differentiated accesses to
several users.
So far we've always used libpq defaults when connecting to workers; bar
special environment variables being set that'll always be the user that
started the server. That's not desirable because it prevents using
users with fewer privileges.
Thus change the various APIs creating connections to workers to always
use usernames. That means:
1) MultiClientConnect() needs to, optionally, accept a username
2) GetOrEstablishConnection(), including the underlying cache, need to
use the current user as part of the connection cache key. That way
connections for separate users are distinct, and we always use one
with the correct authorization.
3) The task tracker needs to keep track of the username associated with
a task, so it can use it when establishing connections outside the
originating session.
This commit adds a fast shard pruning path for INSERTs on
hash-partitioned tables. The rationale behind this change is
that if there exists a sorted shard interval array, a single
index lookup on the array allows us to find the corresponding
shard interval. As mentioned above, we need a sorted
(wrt shardminvalue) shard interval array. Thus, this commit
updates shardIntervalArray to sortedShardIntervalArray in the
metadata cache. Then uses the low-level API that is defined in
multi_copy to handle the fast shard pruning.
The performance impact of this change is more apparent as more
shards exist for a distributed table. Previous implementation
was relying on linear search through the shard intervals. However,
this commit relies on constant lookup time on shard interval
array. Thus, the shard pruning becomes less dependent on the
shard count.
This change fixes the problem with joins with VARCHAR columns. Prior to
this change, when we tried to do large table joins on varchar columns, we got
an error of the form:
ERROR: cannot perform local joins that involve expressions
DETAIL: local joins can be performed between columns only.
This is because we have a check in CheckJoinBetweenColumns() which requires the
join clause to have only 'Var' nodes (i.e. columns). Postgres adds a relabel t
ype cast to cast the varchar to text; hence the type of the node is not T_Var
and the join fails.
The fix involves calling strip_implicit_coercions() to the left and right
arguments so that RELABELTYPE is stripped to VAR.
Fixes#76.
While reading this code last week, it appeared as though there was no
place we ensured that the partition clause actually used equality ops.
As such, I was worried that we might transform a clause such as id < 5
into a constraint like hash(id) = hash(5) when doing shard pruning. The
relevant code seemed to just ensure:
1. The node is an OpExpr
2. With a related hash function
3. It compares the partition column
4. Against a constant
A superficial reading implied we didn't actually make sure the original
op was equality-related, but it turns out the hash lookup function DOES
ensure that for us. So I added a comment.
Fixes issue #258
Prior to this change, Citus gives a deceptive NOTICE message when a query
including ANY or ALL on a non-partition column is issued on a hash
partitioned table.
Let the github_events table be hash-distributed on repo_id column. Then,
issuing this query:
SELECT count(*) FROM github_events WHERE event_id = ANY ('{1,2,3}')
Gives this message:
NOTICE: cannot use shard pruning with ANY (array expression)
HINT: Consider rewriting the expression with OR clauses.
Note that since event_id is not the partition column, shard pruning would
not be applied in any case. However, the NOTICE message would be valid
and be given if the ANY clause would have been applied on repo_id column.
Reviewer: Murat Tuncer