Add initial sql support for distributed tables that don't have a shard key (#6773/#6822)

Enable router planner and a limited version of INSERT .. SELECT planner
for the queries that reference colocated null shard key tables.

* SELECT / UPDATE / DELETE / MERGE is supported as long as it's a router
query.
* INSERT .. SELECT is supported as long as it only references colocated
  null shard key tables.

Note that this is not only limited to distributed INSERT .. SELECT but
also
covers a limited set of query types that require pull-to-coordinator,
e.g.,
  due to LIMIT clause, generate_series() etc. ...
(Ideally distributed INSERT .. SELECT could handle such queries too,
e.g.,
when we're only referencing tables that don't have a shard key, but
today
this is not the case. See
https://github.com/citusdata/citus/pull/6773#discussion_r1140130562.
pull/6867/head
Onur Tirtir 2023-03-28 18:47:59 +03:00
parent ac0ffc9839
commit 85745b46d5
16 changed files with 3375 additions and 21 deletions

View File

@ -1025,6 +1025,17 @@ CreateDistributedPlan(uint64 planId, bool allowRecursivePlanning, Query *origina
{
return distributedPlan;
}
else if (ContainsNullDistKeyTable(originalQuery))
{
/*
* We only support router queries if the query contains reference to
* a null-dist-key table. This temporary restriction will be removed
* once we support recursive planning for the queries that reference
* null-dist-key tables.
*/
WrapRouterErrorForNullDistKeyTable(distributedPlan->planningError);
RaiseDeferredError(distributedPlan->planningError, ERROR);
}
else
{
RaiseDeferredError(distributedPlan->planningError, DEBUG2);
@ -2462,6 +2473,18 @@ HasUnresolvedExternParamsWalker(Node *expression, ParamListInfo boundParams)
}
/*
* ContainsNullDistKeyTable returns true if given query contains reference
* to a null-dist-key table.
*/
bool
ContainsNullDistKeyTable(Query *query)
{
RTEListProperties *rteListProperties = GetRTEListPropertiesForQuery(query);
return rteListProperties->hasDistTableWithoutShardKey;
}
/*
* GetRTEListPropertiesForQuery is a wrapper around GetRTEListProperties that
* returns RTEListProperties for the rte list retrieved from query.
@ -2538,6 +2561,15 @@ GetRTEListProperties(List *rangeTableList)
else if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE))
{
rteListProperties->hasDistributedTable = true;
if (!HasDistributionKeyCacheEntry(cacheEntry))
{
rteListProperties->hasDistTableWithoutShardKey = true;
}
else
{
rteListProperties->hasDistTableWithShardKey = true;
}
}
else
{

View File

@ -212,6 +212,16 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
return false;
}
/*
* If the table doesn't have a distribution column, we don't need to
* check anything further.
*/
Var *distributionKey = PartitionColumn(distributedTableId, 1);
if (!distributionKey)
{
return true;
}
/* WHERE clause should not be empty for distributed tables */
if (joinTree == NULL ||
(IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && joinTree->quals ==
@ -220,13 +230,6 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
return false;
}
/* if that's a reference table, we don't need to check anything further */
Var *distributionKey = PartitionColumn(distributedTableId, 1);
if (!distributionKey)
{
return true;
}
/* convert list of expressions into expression tree for further processing */
quals = joinTree->quals;
if (quals != NULL && IsA(quals, List))

View File

@ -57,6 +57,7 @@ static DistributedPlan * CreateInsertSelectPlanInternal(uint64 planId,
PlannerRestrictionContext *
plannerRestrictionContext,
ParamListInfo boundParams);
static void ErrorIfInsertSelectWithNullDistKeyNotSupported(Query *originalQuery);
static DistributedPlan * CreateDistributedInsertSelectPlan(Query *originalQuery,
PlannerRestrictionContext *
plannerRestrictionContext);
@ -241,6 +242,12 @@ CreateInsertSelectPlanInternal(uint64 planId, Query *originalQuery,
RaiseDeferredError(deferredError, ERROR);
}
/*
* We support a limited set of INSERT .. SELECT queries if the query
* references a null-dist-key table.
*/
ErrorIfInsertSelectWithNullDistKeyNotSupported(originalQuery);
DistributedPlan *distributedPlan = CreateDistributedInsertSelectPlan(originalQuery,
plannerRestrictionContext);
@ -260,6 +267,74 @@ CreateInsertSelectPlanInternal(uint64 planId, Query *originalQuery,
}
/*
* ErrorIfInsertSelectWithNullDistKeyNotSupported throws an error if given INSERT
* .. SELECT query references a null-dist-key table (as the target table or in
* the SELECT clause) and is unsupported.
*
* Such an INSERT .. SELECT query is supported as long as the it only references
* a "colocated" set of null-dist-key tables, no other relation rte types.
*/
static void
ErrorIfInsertSelectWithNullDistKeyNotSupported(Query *originalQuery)
{
RangeTblEntry *subqueryRte = ExtractSelectRangeTableEntry(originalQuery);
Query *subquery = subqueryRte->subquery;
RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery(subquery);
RangeTblEntry *insertRte = ExtractResultRelationRTEOrError(originalQuery);
Oid targetRelationId = insertRte->relid;
if (!IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE) &&
subqueryRteListProperties->hasDistTableWithoutShardKey)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot select from a distributed table that "
"does not have a shard key when inserting into "
"a different table type")));
}
else if (IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE))
{
if (subqueryRteListProperties->hasPostgresLocalTable ||
subqueryRteListProperties->hasReferenceTable ||
subqueryRteListProperties->hasCitusLocalTable ||
subqueryRteListProperties->hasDistTableWithShardKey ||
subqueryRteListProperties->hasMaterializedView)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot select from different table types "
"when inserting into a distributed table "
"that does not have a shard key")));
}
if (!subqueryRteListProperties->hasDistTableWithoutShardKey)
{
/*
* This means that the SELECT doesn't reference any Citus tables,
* Postgres tables or materialized views but references a function
* call, a values claue etc., or a cte from INSERT.
*
* In that case, we rely on the common restrictions enforced by the
* INSERT .. SELECT planners.
*/
Assert(!NeedsDistributedPlanning(subquery));
return;
}
List *distributedRelationIdList = DistributedRelationIdList(subquery);
distributedRelationIdList = lappend_oid(distributedRelationIdList,
targetRelationId);
if (!AllDistributedRelationsInListColocated(distributedRelationIdList))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot select from a non-colocated distributed "
"table when inserting into a distributed table "
"that does not have a shard key")));
}
}
}
/*
* CreateDistributedInsertSelectPlan creates a DistributedPlan for distributed
* INSERT ... SELECT queries which could consist of multiple tasks.
@ -379,6 +454,16 @@ CreateInsertSelectIntoLocalTablePlan(uint64 planId, Query *insertSelectQuery,
{
RangeTblEntry *selectRte = ExtractSelectRangeTableEntry(insertSelectQuery);
RTEListProperties *selectRteListProperties =
GetRTEListPropertiesForQuery(selectRte->subquery);
if (selectRteListProperties->hasDistTableWithoutShardKey)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot select from a distributed table that "
"does not have a shard key when inserting into "
"a local table")));
}
PrepareInsertSelectForCitusPlanner(insertSelectQuery);
/* get the SELECT query (may have changed after PrepareInsertSelectForCitusPlanner) */
@ -717,10 +802,7 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte,
}
else if (IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"distributed INSERT ... SELECT cannot target a distributed "
"table with a null shard key",
NULL, NULL);
/* we've already checked the subquery via ErrorIfInsertSelectWithNullDistKeyNotSupported */
}
else
{
@ -874,7 +956,7 @@ RouterModifyTaskForShardInterval(Query *originalQuery,
*/
RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery(
copiedSubquery);
if (subqueryRteListProperties->hasDistributedTable)
if (subqueryRteListProperties->hasDistTableWithShardKey)
{
AddPartitionKeyNotNullFilterToSelect(copiedSubquery);
}

View File

@ -509,6 +509,11 @@ InsertDistributionColumnMatchesSource(Oid targetRelationId, Query *query)
return NULL;
}
if (!HasDistributionKey(targetRelationId))
{
return NULL;
}
bool foundDistributionColumn = false;
MergeAction *action = NULL;
foreach_ptr(action, query->mergeActionList)

View File

@ -272,7 +272,7 @@ TargetListOnPartitionColumn(Query *query, List *targetEntryList)
if (!targetListOnPartitionColumn)
{
if (!FindNodeMatchingCheckFunctionInRangeTableList(query->rtable,
IsDistributedTableRTE))
IsTableWithDistKeyRTE))
{
targetListOnPartitionColumn = true;
}
@ -379,6 +379,20 @@ IsReferenceTableRTE(Node *node)
}
/*
* IsTableWithDistKeyRTE gets a node and returns true if the node
* is a range table relation entry that points to a distributed table
* that has a distribution column.
*/
bool
IsTableWithDistKeyRTE(Node *node)
{
Oid relationId = NodeTryGetRteRelid(node);
return relationId != InvalidOid && IsCitusTable(relationId) &&
HasDistributionKey(relationId);
}
/*
* FullCompositeFieldList gets a composite field list, and checks if all fields
* of composite type are used in the list.

View File

@ -258,6 +258,22 @@ CreateModifyPlan(Query *originalQuery, Query *query,
}
/*
* WrapRouterErrorForNullDistKeyTable wraps given planning error with a
* generic error message if given query references a distributed table
* that doesn't have a distribution key.
*/
void
WrapRouterErrorForNullDistKeyTable(DeferredErrorMessage *planningError)
{
planningError->detail = planningError->message;
planningError->message = pstrdup("queries that reference a distributed "
"table without a shard key can only "
"reference colocated distributed "
"tables or reference tables");
}
/*
* CreateSingleTaskRouterSelectPlan creates a physical plan for given SELECT query.
* The returned plan is a router task that returns query results from a single worker.
@ -1870,6 +1886,11 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
*/
if (IsMergeQuery(originalQuery))
{
if (ContainsNullDistKeyTable(originalQuery))
{
WrapRouterErrorForNullDistKeyTable(*planningError);
}
RaiseDeferredError(*planningError, ERROR);
}
else
@ -3855,7 +3876,8 @@ ErrorIfQueryHasUnroutableModifyingCTE(Query *queryTree)
CitusTableCacheEntry *modificationTableCacheEntry =
GetCitusTableCacheEntry(distributedTableId);
if (!HasDistributionKeyCacheEntry(modificationTableCacheEntry))
if (!IsCitusTableTypeCacheEntry(modificationTableCacheEntry,
DISTRIBUTED_TABLE))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"cannot router plan modification of a non-distributed table",

View File

@ -147,9 +147,19 @@ typedef struct RTEListProperties
bool hasReferenceTable;
bool hasCitusLocalTable;
/* includes hash, append and range partitioned tables */
/* includes hash, null dist key, append and range partitioned tables */
bool hasDistributedTable;
/*
* Effectively, hasDistributedTable is equal to
* "hasDistTableWithShardKey || hasDistTableWithoutShardKey".
*
* We provide below two for the callers that want to know what kind of
* distributed tables that given query has references to.
*/
bool hasDistTableWithShardKey;
bool hasDistTableWithoutShardKey;
/* union of hasReferenceTable, hasCitusLocalTable and hasDistributedTable */
bool hasCitusTable;
@ -243,6 +253,7 @@ extern int32 BlessRecordExpression(Expr *expr);
extern void DissuadePlannerFromUsingPlan(PlannedStmt *plan);
extern PlannedStmt * FinalizePlan(PlannedStmt *localPlan,
struct DistributedPlan *distributedPlan);
extern bool ContainsNullDistKeyTable(Query *query);
extern RTEListProperties * GetRTEListPropertiesForQuery(Query *query);

View File

@ -200,6 +200,7 @@ extern bool IsCitusTableRTE(Node *node);
extern bool IsDistributedOrReferenceTableRTE(Node *node);
extern bool IsDistributedTableRTE(Node *node);
extern bool IsReferenceTableRTE(Node *node);
extern bool IsTableWithDistKeyRTE(Node *node);
extern bool IsCitusExtraDataContainerRelation(RangeTblEntry *rte);
extern bool ContainsReadIntermediateResultFunction(Node *node);
extern bool ContainsReadIntermediateResultArrayFunction(Node *node);

View File

@ -36,6 +36,7 @@ extern DistributedPlan * CreateRouterPlan(Query *originalQuery, Query *query,
extern DistributedPlan * CreateModifyPlan(Query *originalQuery, Query *query,
PlannerRestrictionContext *
plannerRestrictionContext);
extern void WrapRouterErrorForNullDistKeyTable(DeferredErrorMessage *planningError);
extern DeferredErrorMessage * PlanRouterQuery(Query *originalQuery,
PlannerRestrictionContext *
plannerRestrictionContext,

View File

@ -237,12 +237,6 @@ class AllNullDistKeyDefaultConfig(CitusDefaultClusterConfig):
# group 8
"function_create",
"functions",
# group 9
"merge_arbitrary_create",
"merge_arbitrary",
# group 10
"arbitrary_configs_router_create",
"arbitrary_configs_router",
#
# ii) Skip the following test as it requires support for create_distributed_function.
"nested_execution",

View File

@ -1803,6 +1803,7 @@ ALTER TABLE trigger_table_3 ENABLE TRIGGER ALL;
-- try a few simple queries at least to make sure that we don't crash
BEGIN;
INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1;
ERROR: cannot select from a non-colocated distributed table when inserting into a distributed table that does not have a shard key
ROLLBACK;
DROP TRIGGER IF EXISTS trigger_1 ON trigger_table_1;
DROP TRIGGER trigger_2 ON trigger_table_2 CASCADE;

View File

@ -3228,6 +3228,153 @@ WHEN NOT MATCHED THEN
INSERT VALUES(dist_source.id, dist_source.val);
ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported
HINT: Consider using hash distribution instead
-- test merge with null shard key tables
CREATE SCHEMA query_null_dist_key;
SET search_path TO query_null_dist_key;
SET client_min_messages TO DEBUG2;
CREATE TABLE nullkey_c1_t1(a int, b int);
CREATE TABLE nullkey_c1_t2(a int, b int);
SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE nullkey_c2_t1(a int, b int);
CREATE TABLE nullkey_c2_t2(a int, b int);
SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table(a int, b int);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
CREATE TABLE distributed_table(a int, b int);
SELECT create_distributed_table('distributed_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
CREATE TABLE citus_local_table(a int, b int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
CREATE TABLE postgres_local_table(a int, b int);
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i;
-- with a colocated table
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b;
DEBUG: <Deparsed MERGE query: MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING query_null_dist_key.nullkey_c1_t2_4000146 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b>
DEBUG: Creating MERGE router plan
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN DELETE;
DEBUG: <Deparsed MERGE query: MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING query_null_dist_key.nullkey_c1_t2_4000146 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE>
DEBUG: Creating MERGE router plan
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
DEBUG: <Deparsed MERGE query: MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING query_null_dist_key.nullkey_c1_t2_4000146 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
DEBUG: Creating MERGE router plan
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
DEBUG: <Deparsed MERGE query: MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING query_null_dist_key.nullkey_c1_t2_4000146 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
DEBUG: Creating MERGE router plan
-- with non-colocated null-dist-key table
MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b;
ERROR: For MERGE command, all the distributed tables must be colocated
MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, nullkey_c2_t1.b);
ERROR: For MERGE command, all the distributed tables must be colocated
-- with a distributed table
MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a)
WHEN MATCHED THEN UPDATE SET b = distributed_table.b
WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, distributed_table.b);
ERROR: For MERGE command, all the distributed tables must be colocated
MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a)
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
ERROR: For MERGE command, all the distributed tables must be colocated
-- with a reference table
MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a)
WHEN MATCHED THEN UPDATE SET b = reference_table.b;
ERROR: MERGE command is not supported on reference tables yet
MERGE INTO reference_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = reference_table.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
ERROR: MERGE command is not supported on reference tables yet
-- with a citus local table
MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a)
WHEN MATCHED THEN UPDATE SET b = citus_local_table.b;
ERROR: MERGE command is not supported with combination of distributed/local tables yet
MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a)
WHEN MATCHED THEN DELETE;
ERROR: MERGE command is not supported with combination of distributed/local tables yet
-- with a postgres table
MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a)
WHEN MATCHED THEN UPDATE SET b = postgres_local_table.b;
ERROR: MERGE command is not supported with combination of distributed/local tables yet
MERGE INTO postgres_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = postgres_local_table.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
ERROR: MERGE command is not supported with combination of distributed/local tables yet
-- using ctes
WITH cte AS (
SELECT * FROM nullkey_c1_t1
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
DEBUG: <Deparsed MERGE query: WITH cte AS (SELECT nullkey_c1_t1_1.a, nullkey_c1_t1_1.b FROM query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1_1) MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b>
DEBUG: Creating MERGE router plan
WITH cte AS (
SELECT * FROM distributed_table
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
ERROR: For MERGE command, all the distributed tables must be colocated
WITH cte AS materialized (
SELECT * FROM distributed_table
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
ERROR: For MERGE command, all the distributed tables must be colocated
SET client_min_messages TO WARNING;
DROP SCHEMA query_null_dist_key CASCADE;
RESET client_min_messages;
SET search_path TO merge_schema;
DROP SERVER foreign_server CASCADE;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to user mapping for postgres on server foreign_server

File diff suppressed because it is too large Load Diff

View File

@ -200,6 +200,7 @@ test: local_table_join
test: local_dist_join_mixed
test: citus_local_dist_joins
test: recurring_outer_join
test: query_null_dist_key
test: pg_dump
# ---------

View File

@ -2051,6 +2051,118 @@ UPDATE SET val = dist_source.val
WHEN NOT MATCHED THEN
INSERT VALUES(dist_source.id, dist_source.val);
-- test merge with null shard key tables
CREATE SCHEMA query_null_dist_key;
SET search_path TO query_null_dist_key;
SET client_min_messages TO DEBUG2;
CREATE TABLE nullkey_c1_t1(a int, b int);
CREATE TABLE nullkey_c1_t2(a int, b int);
SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none');
SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1');
CREATE TABLE nullkey_c2_t1(a int, b int);
CREATE TABLE nullkey_c2_t2(a int, b int);
SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none');
SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null);
CREATE TABLE reference_table(a int, b int);
SELECT create_reference_table('reference_table');
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
CREATE TABLE distributed_table(a int, b int);
SELECT create_distributed_table('distributed_table', 'a');
INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i;
CREATE TABLE citus_local_table(a int, b int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i;
CREATE TABLE postgres_local_table(a int, b int);
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i;
-- with a colocated table
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b;
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN DELETE;
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
-- with non-colocated null-dist-key table
MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b;
MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, nullkey_c2_t1.b);
-- with a distributed table
MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a)
WHEN MATCHED THEN UPDATE SET b = distributed_table.b
WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, distributed_table.b);
MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a)
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
-- with a reference table
MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a)
WHEN MATCHED THEN UPDATE SET b = reference_table.b;
MERGE INTO reference_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = reference_table.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
-- with a citus local table
MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a)
WHEN MATCHED THEN UPDATE SET b = citus_local_table.b;
MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a)
WHEN MATCHED THEN DELETE;
-- with a postgres table
MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a)
WHEN MATCHED THEN UPDATE SET b = postgres_local_table.b;
MERGE INTO postgres_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = postgres_local_table.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
-- using ctes
WITH cte AS (
SELECT * FROM nullkey_c1_t1
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
WITH cte AS (
SELECT * FROM distributed_table
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
WITH cte AS materialized (
SELECT * FROM distributed_table
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
SET client_min_messages TO WARNING;
DROP SCHEMA query_null_dist_key CASCADE;
RESET client_min_messages;
SET search_path TO merge_schema;
DROP SERVER foreign_server CASCADE;
DROP FUNCTION merge_when_and_write();
DROP SCHEMA merge_schema CASCADE;

File diff suppressed because it is too large Load Diff