mirror of https://github.com/citusdata/citus.git
When replication factor > 1, all modifications are done via 2PC (#5379)
With Citus 9.0, we introduced `citus.single_shard_commit_protocol` which defaults to 2PC. With this commit, we prevent any user to set it to 1PC and drop support for `citus.single_shard_commit_protocol`. Although this might add some overhead for users, it is already the default behaviour (so less likely) and marking placements as INVALID is much worse.decrease/resourceSize
parent
a851211dbc
commit
3f726c72e0
|
@ -1416,34 +1416,12 @@ DistributedExecutionRequiresRollback(List *taskList)
|
||||||
|
|
||||||
if (list_length(task->taskPlacementList) > 1)
|
if (list_length(task->taskPlacementList) > 1)
|
||||||
{
|
{
|
||||||
if (SingleShardCommitProtocol == COMMIT_PROTOCOL_2PC)
|
|
||||||
{
|
|
||||||
/*
|
|
||||||
* Adaptive executor opts to error out on queries if a placement is unhealthy,
|
|
||||||
* not marking the placement itself unhealthy in the process.
|
|
||||||
* Use 2PC to rollback placements before the unhealthy replica failed.
|
|
||||||
*/
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Some tasks don't set replicationModel thus we only
|
* Single DML/DDL tasks with replicated tables (including
|
||||||
* rely on the anchorShardId, not replicationModel.
|
* reference and non-reference tables) should require
|
||||||
*
|
* BEGIN/COMMIT/ROLLBACK.
|
||||||
* TODO: Do we ever need replicationModel in the Task structure?
|
|
||||||
* Can't we always rely on anchorShardId?
|
|
||||||
*/
|
*/
|
||||||
if (task->anchorShardId != INVALID_SHARD_ID && ReferenceTableShardId(
|
return true;
|
||||||
task->anchorShardId))
|
|
||||||
{
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Single DML/DDL tasks with replicated tables (non-reference)
|
|
||||||
* should not require BEGIN/COMMIT/ROLLBACK.
|
|
||||||
*/
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
|
@ -1467,21 +1445,13 @@ TaskListRequires2PC(List *taskList)
|
||||||
}
|
}
|
||||||
|
|
||||||
Task *task = (Task *) linitial(taskList);
|
Task *task = (Task *) linitial(taskList);
|
||||||
if (task->replicationModel == REPLICATION_MODEL_2PC)
|
if (list_length(task->taskPlacementList) > 1)
|
||||||
{
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Some tasks don't set replicationModel thus we rely on
|
|
||||||
* the anchorShardId as well replicationModel.
|
|
||||||
*
|
|
||||||
* TODO: Do we ever need replicationModel in the Task structure?
|
|
||||||
* Can't we always rely on anchorShardId?
|
|
||||||
*/
|
|
||||||
uint64 anchorShardId = task->anchorShardId;
|
|
||||||
if (anchorShardId != INVALID_SHARD_ID && ReferenceTableShardId(anchorShardId))
|
|
||||||
{
|
{
|
||||||
|
/*
|
||||||
|
* Even single DML/DDL tasks with replicated tables
|
||||||
|
* (including reference and non-reference tables)
|
||||||
|
* should require BEGIN/COMMIT/ROLLBACK.
|
||||||
|
*/
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -298,6 +298,47 @@ EnsureModificationsCanRun(void)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureModificationsCanRunOnRelation firsts calls into EnsureModificationsCanRun() and
|
||||||
|
* then does one more additional check. The additional check is to give a proper error
|
||||||
|
* message if any relation that is modified is replicated, as replicated tables use
|
||||||
|
* 2PC and 2PC cannot happen when recovery is in progress.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
EnsureModificationsCanRunOnRelation(Oid relationId)
|
||||||
|
{
|
||||||
|
EnsureModificationsCanRun();
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Even if user allows writes from standby, we should not allow for
|
||||||
|
* replicated tables as they require 2PC. And, 2PC needs to write a log
|
||||||
|
* record on the coordinator.
|
||||||
|
*/
|
||||||
|
if (!(RecoveryInProgress() && WritableStandbyCoordinator))
|
||||||
|
{
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!IsCitusTable(relationId))
|
||||||
|
{
|
||||||
|
/* we are not interested in PG tables */
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (IsCitusTableType(relationId, REFERENCE_TABLE) ||
|
||||||
|
!SingleReplicatedTable(relationId))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("writing to worker nodes is not currently "
|
||||||
|
"allowed for replicated tables such as reference "
|
||||||
|
"tables or hash distributed tables with replication "
|
||||||
|
"factor greater than 1."),
|
||||||
|
errhint("All modifications to replicated tables happen via 2PC, "
|
||||||
|
"and 2PC requires the database to be in a writable state."),
|
||||||
|
errdetail("the database is read-only")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* IsCitusTableType returns true if the given table with relationId
|
* IsCitusTableType returns true if the given table with relationId
|
||||||
* belongs to a citus table that matches the given table type. If cache
|
* belongs to a citus table that matches the given table type. If cache
|
||||||
|
|
|
@ -847,9 +847,10 @@ CreateDistributedPlan(uint64 planId, Query *originalQuery, Query *query, ParamLi
|
||||||
|
|
||||||
if (IsModifyCommand(originalQuery))
|
if (IsModifyCommand(originalQuery))
|
||||||
{
|
{
|
||||||
EnsureModificationsCanRun();
|
|
||||||
|
|
||||||
Oid targetRelationId = ModifyQueryResultRelationId(query);
|
Oid targetRelationId = ModifyQueryResultRelationId(query);
|
||||||
|
|
||||||
|
EnsureModificationsCanRunOnRelation(targetRelationId);
|
||||||
|
|
||||||
EnsurePartitionTableNotReplicated(targetRelationId);
|
EnsurePartitionTableNotReplicated(targetRelationId);
|
||||||
|
|
||||||
if (InsertSelectIntoCitusTable(originalQuery))
|
if (InsertSelectIntoCitusTable(originalQuery))
|
||||||
|
|
|
@ -1643,21 +1643,6 @@ RegisterCitusConfigVariables(void)
|
||||||
GUC_STANDARD,
|
GUC_STANDARD,
|
||||||
NULL, NULL, NULL);
|
NULL, NULL, NULL);
|
||||||
|
|
||||||
DefineCustomEnumVariable(
|
|
||||||
"citus.single_shard_commit_protocol",
|
|
||||||
gettext_noop(
|
|
||||||
"Sets the commit protocol for commands modifying a single shards with multiple replicas."),
|
|
||||||
gettext_noop("When a failure occurs during commands that modify multiple "
|
|
||||||
"replicas, two-phase commit is required to ensure data is never lost "
|
|
||||||
"and this is the default. However, changing to 1pc may give small "
|
|
||||||
"performance benefits."),
|
|
||||||
&SingleShardCommitProtocol,
|
|
||||||
COMMIT_PROTOCOL_2PC,
|
|
||||||
shard_commit_protocol_options,
|
|
||||||
PGC_USERSET,
|
|
||||||
GUC_NO_SHOW_ALL,
|
|
||||||
NULL, NULL, NULL);
|
|
||||||
|
|
||||||
DefineCustomBoolVariable(
|
DefineCustomBoolVariable(
|
||||||
"citus.sort_returning",
|
"citus.sort_returning",
|
||||||
gettext_noop("Sorts the RETURNING clause to get consistent test output"),
|
gettext_noop("Sorts the RETURNING clause to get consistent test output"),
|
||||||
|
|
|
@ -48,7 +48,6 @@ CoordinatedTransactionState CurrentCoordinatedTransactionState = COORD_TRANS_NON
|
||||||
|
|
||||||
/* GUC, the commit protocol to use for commands affecting more than one connection */
|
/* GUC, the commit protocol to use for commands affecting more than one connection */
|
||||||
int MultiShardCommitProtocol = COMMIT_PROTOCOL_2PC;
|
int MultiShardCommitProtocol = COMMIT_PROTOCOL_2PC;
|
||||||
int SingleShardCommitProtocol = COMMIT_PROTOCOL_2PC;
|
|
||||||
int SavedMultiShardCommitProtocol = COMMIT_PROTOCOL_BARE;
|
int SavedMultiShardCommitProtocol = COMMIT_PROTOCOL_BARE;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
|
@ -196,6 +196,7 @@ extern bool InstalledAndAvailableVersionsSame(void);
|
||||||
extern bool MajorVersionsCompatible(char *leftVersion, char *rightVersion);
|
extern bool MajorVersionsCompatible(char *leftVersion, char *rightVersion);
|
||||||
extern void ErrorIfInconsistentShardIntervals(CitusTableCacheEntry *cacheEntry);
|
extern void ErrorIfInconsistentShardIntervals(CitusTableCacheEntry *cacheEntry);
|
||||||
extern void EnsureModificationsCanRun(void);
|
extern void EnsureModificationsCanRun(void);
|
||||||
|
extern void EnsureModificationsCanRunOnRelation(Oid relationId);
|
||||||
extern char LookupDistributionMethod(Oid distributionMethodOid);
|
extern char LookupDistributionMethod(Oid distributionMethodOid);
|
||||||
extern bool RelationExists(Oid relationId);
|
extern bool RelationExists(Oid relationId);
|
||||||
extern ShardInterval * TupleToShardInterval(HeapTuple heapTuple,
|
extern ShardInterval * TupleToShardInterval(HeapTuple heapTuple,
|
||||||
|
|
|
@ -119,6 +119,8 @@ s/(ERROR.*)pgsql_job_cache\/([0-9]+_[0-9]+_[0-9]+)\/(.*).data/\1pgsql_job_cache\
|
||||||
|
|
||||||
# assign_distributed_transaction id params
|
# assign_distributed_transaction id params
|
||||||
s/(NOTICE.*)assign_distributed_transaction_id\([0-9]+, [0-9]+, '.*'\)/\1assign_distributed_transaction_id\(xx, xx, 'xxxxxxx'\)/g
|
s/(NOTICE.*)assign_distributed_transaction_id\([0-9]+, [0-9]+, '.*'\)/\1assign_distributed_transaction_id\(xx, xx, 'xxxxxxx'\)/g
|
||||||
|
s/(NOTICE.*)PREPARE TRANSACTION 'citus_[0-9]+_[0-9]+_[0-9]+_[0-9]+'/\1PREPARE TRANSACTION 'citus_xx_xx_xx_xx'/g
|
||||||
|
s/(NOTICE.*)COMMIT PREPARED 'citus_[0-9]+_[0-9]+_[0-9]+_[0-9]+'/\1COMMIT PREPARED 'citus_xx_xx_xx_xx'/g
|
||||||
|
|
||||||
# toast tables
|
# toast tables
|
||||||
s/pg_toast_[0-9]+/pg_toast_xxxxx/g
|
s/pg_toast_[0-9]+/pg_toast_xxxxx/g
|
||||||
|
|
|
@ -174,9 +174,9 @@ SELECT citus.mitmproxy('conn.delay(500)');
|
||||||
SELECT count(*) FROM single_replicatated;
|
SELECT count(*) FROM single_replicatated;
|
||||||
ERROR: could not establish any connections to the node localhost:xxxxx after 400 ms
|
ERROR: could not establish any connections to the node localhost:xxxxx after 400 ms
|
||||||
SET citus.force_max_query_parallelization TO OFF;
|
SET citus.force_max_query_parallelization TO OFF;
|
||||||
-- one similar test, but this time on modification queries
|
-- one similar test, and this time on modification queries
|
||||||
-- to see that connection establishement failures could
|
-- to see that connection establishement failures could
|
||||||
-- mark placement INVALID
|
-- fail the transaction (but not mark any placements as INVALID)
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -203,6 +203,7 @@ SELECT citus.mitmproxy('conn.delay(500)');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
INSERT INTO products VALUES (100, '100', 100);
|
INSERT INTO products VALUES (100, '100', 100);
|
||||||
|
ERROR: could not establish any connections to the node localhost:xxxxx after 400 ms
|
||||||
COMMIT;
|
COMMIT;
|
||||||
SELECT
|
SELECT
|
||||||
count(*) as invalid_placement_count
|
count(*) as invalid_placement_count
|
||||||
|
@ -213,14 +214,14 @@ WHERE
|
||||||
shardid IN (SELECT shardid from pg_dist_shard where logicalrelid = 'products'::regclass);
|
shardid IN (SELECT shardid from pg_dist_shard where logicalrelid = 'products'::regclass);
|
||||||
invalid_placement_count
|
invalid_placement_count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- show that INSERT went through
|
-- show that INSERT failed
|
||||||
SELECT count(*) FROM products WHERE product_no = 100;
|
SELECT count(*) FROM products WHERE product_no = 100;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
RESET client_min_messages;
|
RESET client_min_messages;
|
||||||
|
|
|
@ -374,7 +374,7 @@ SELECT create_distributed_table('dml_test', 'id');
|
||||||
|
|
||||||
COPY dml_test FROM STDIN WITH CSV;
|
COPY dml_test FROM STDIN WITH CSV;
|
||||||
---- test multiple statements against a single shard, but with two placements
|
---- test multiple statements against a single shard, but with two placements
|
||||||
-- fail at COMMIT (actually COMMIT this time, as no 2pc in use)
|
-- fail at PREPARED COMMIT as we use 2PC
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -400,7 +400,27 @@ WARNING: server closed the connection unexpectedly
|
||||||
connection not open
|
connection not open
|
||||||
connection not open
|
connection not open
|
||||||
CONTEXT: while executing command on localhost:xxxxx
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
--- should see all changes, but they only went to one placement (other is unhealthy)
|
-- all changes should be committed because we injected
|
||||||
|
-- the failure on the COMMIT time. And, we should not
|
||||||
|
-- mark any placements as INVALID
|
||||||
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
|
mitmproxy
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT recover_prepared_transactions();
|
||||||
|
recover_prepared_transactions
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT shardid FROM pg_dist_shard_placement WHERE shardstate = 3;
|
||||||
|
shardid
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
(0 rows)
|
||||||
|
|
||||||
|
SET citus.task_assignment_policy TO "round-robin";
|
||||||
SELECT * FROM dml_test ORDER BY id ASC;
|
SELECT * FROM dml_test ORDER BY id ASC;
|
||||||
id | name
|
id | name
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -409,18 +429,15 @@ SELECT * FROM dml_test ORDER BY id ASC;
|
||||||
5 | Epsilon
|
5 | Epsilon
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
SELECT shardid FROM pg_dist_shard_placement WHERE shardstate = 3;
|
SELECT * FROM dml_test ORDER BY id ASC;
|
||||||
shardid
|
id | name
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
103402
|
3 | gamma
|
||||||
(1 row)
|
4 | Delta
|
||||||
|
5 | Epsilon
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
(3 rows)
|
||||||
mitmproxy
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
|
RESET citus.task_assignment_policy;
|
||||||
-- drop table and recreate as reference table
|
-- drop table and recreate as reference table
|
||||||
DROP TABLE dml_test;
|
DROP TABLE dml_test;
|
||||||
SET citus.shard_count = 2;
|
SET citus.shard_count = 2;
|
||||||
|
|
|
@ -28,7 +28,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
INSERT INTO partitioned_table VALUES (0, 0);
|
INSERT INTO partitioned_table VALUES (0, 0);
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
||||||
This probably means the server terminated abnormally
|
This probably means the server terminated abnormally
|
||||||
before or while processing the request.
|
before or while processing the request.
|
||||||
-- use both placements
|
-- use both placements
|
||||||
|
@ -37,25 +37,25 @@ SET citus.task_assignment_policy TO "round-robin";
|
||||||
SELECT count(*) FROM partitioned_table_0;
|
SELECT count(*) FROM partitioned_table_0;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
2
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT count(*) FROM partitioned_table_0;
|
SELECT count(*) FROM partitioned_table_0;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
2
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT count(*) FROM partitioned_table;
|
SELECT count(*) FROM partitioned_table;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
2
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT count(*) FROM partitioned_table;
|
SELECT count(*) FROM partitioned_table;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
2
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- ==== Clean up, we're done here ====
|
-- ==== Clean up, we're done here ====
|
||||||
|
|
|
@ -27,27 +27,26 @@ SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
INSERT INTO mod_test VALUES (2, 6);
|
INSERT INTO mod_test VALUES (2, 6);
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
||||||
This probably means the server terminated abnormally
|
This probably means the server terminated abnormally
|
||||||
before or while processing the request.
|
before or while processing the request.
|
||||||
SELECT COUNT(*) FROM mod_test WHERE key=2;
|
SELECT COUNT(*) FROM mod_test WHERE key=2;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- some clean up
|
-- none of the placements are marked as INACTIVE
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
UPDATE pg_dist_shard_placement SET shardstate = 1
|
||||||
WHERE shardid IN (
|
WHERE shardid IN (
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
||||||
) AND shardstate = 3 RETURNING placementid;
|
) AND shardstate = 3 RETURNING placementid;
|
||||||
placementid
|
placementid
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
125
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
TRUNCATE mod_test;
|
TRUNCATE mod_test;
|
||||||
-- verify behavior of UPDATE ... RETURNING; should mark as failed
|
-- verify behavior of UPDATE ... RETURNING; should fail the transaction
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -62,33 +61,27 @@ SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
UPDATE mod_test SET value='ok' WHERE key=2 RETURNING key;
|
UPDATE mod_test SET value='ok' WHERE key=2 RETURNING key;
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
||||||
This probably means the server terminated abnormally
|
This probably means the server terminated abnormally
|
||||||
before or while processing the request.
|
before or while processing the request.
|
||||||
key
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
2
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT COUNT(*) FROM mod_test WHERE value='ok';
|
SELECT COUNT(*) FROM mod_test WHERE value='ok';
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- some clean up
|
-- none of the placements are marked as INACTIVE
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
UPDATE pg_dist_shard_placement SET shardstate = 1
|
||||||
WHERE shardid IN (
|
WHERE shardid IN (
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
||||||
) AND shardstate = 3 RETURNING placementid;
|
) AND shardstate = 3 RETURNING placementid;
|
||||||
placementid
|
placementid
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
125
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
TRUNCATE mod_test;
|
TRUNCATE mod_test;
|
||||||
-- verify behavior of multi-statement modifications to a single shard
|
-- verify behavior of multi-statement modifications to a single shard
|
||||||
-- should succeed but mark a placement as failed
|
-- should fail the transaction and never mark placements inactive
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -100,25 +93,24 @@ INSERT INTO mod_test VALUES (2, 6);
|
||||||
INSERT INTO mod_test VALUES (2, 7);
|
INSERT INTO mod_test VALUES (2, 7);
|
||||||
DELETE FROM mod_test WHERE key=2 AND value = '7';
|
DELETE FROM mod_test WHERE key=2 AND value = '7';
|
||||||
UPDATE mod_test SET value='ok' WHERE key=2;
|
UPDATE mod_test SET value='ok' WHERE key=2;
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
||||||
This probably means the server terminated abnormally
|
This probably means the server terminated abnormally
|
||||||
before or while processing the request.
|
before or while processing the request.
|
||||||
COMMIT;
|
COMMIT;
|
||||||
SELECT COUNT(*) FROM mod_test WHERE key=2;
|
SELECT COUNT(*) FROM mod_test WHERE key=2;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- some clean up
|
-- none of the placements are marked as INACTIVE
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
UPDATE pg_dist_shard_placement SET shardstate = 1
|
||||||
WHERE shardid IN (
|
WHERE shardid IN (
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
||||||
) AND shardstate = 3 RETURNING placementid;
|
) AND shardstate = 3 RETURNING placementid;
|
||||||
placementid
|
placementid
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
125
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
TRUNCATE mod_test;
|
TRUNCATE mod_test;
|
||||||
-- ==== Clean up, we're done here ====
|
-- ==== Clean up, we're done here ====
|
||||||
|
|
|
@ -45,7 +45,8 @@ WARNING: connection to the remote node localhost:xxxxx failed with the followin
|
||||||
3 | test data
|
3 | test data
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- kill after first SELECT; txn should work (though placement marked bad)
|
-- kill after first SELECT; txn should fail as INSERT triggers
|
||||||
|
-- 2PC (and placementis not marked bad)
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").kill()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -55,33 +56,10 @@ SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").kill()');
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO select_test VALUES (3, 'more data');
|
INSERT INTO select_test VALUES (3, 'more data');
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
SELECT * FROM select_test WHERE key = 3;
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
||||||
This probably means the server terminated abnormally
|
This probably means the server terminated abnormally
|
||||||
before or while processing the request.
|
before or while processing the request.
|
||||||
key | value
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
3 | test data
|
|
||||||
3 | more data
|
|
||||||
(2 rows)
|
|
||||||
|
|
||||||
INSERT INTO select_test VALUES (3, 'even more data');
|
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
|
||||||
This probably means the server terminated abnormally
|
|
||||||
before or while processing the request.
|
|
||||||
key | value
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
3 | test data
|
|
||||||
3 | more data
|
|
||||||
3 | even more data
|
|
||||||
(3 rows)
|
|
||||||
|
|
||||||
COMMIT;
|
COMMIT;
|
||||||
-- some clean up
|
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
|
||||||
WHERE shardid IN (
|
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'select_test'::regclass
|
|
||||||
);
|
|
||||||
TRUNCATE select_test;
|
TRUNCATE select_test;
|
||||||
-- now the same tests with query cancellation
|
-- now the same tests with query cancellation
|
||||||
-- put data in shard for which mitm node is first placement
|
-- put data in shard for which mitm node is first placement
|
||||||
|
@ -139,7 +117,7 @@ INSERT INTO select_test VALUES (3, 'even more data');
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
SELECT * FROM select_test WHERE key = 3;
|
||||||
ERROR: canceling statement due to user request
|
ERROR: canceling statement due to user request
|
||||||
COMMIT;
|
COMMIT;
|
||||||
-- error after second SELECT; txn should work (though placement marked bad)
|
-- error after second SELECT; txn should fails the transaction
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(1).reset()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(1).reset()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -156,15 +134,9 @@ SELECT * FROM select_test WHERE key = 3;
|
||||||
|
|
||||||
INSERT INTO select_test VALUES (3, 'even more data');
|
INSERT INTO select_test VALUES (3, 'even more data');
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
SELECT * FROM select_test WHERE key = 3;
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
||||||
This probably means the server terminated abnormally
|
This probably means the server terminated abnormally
|
||||||
before or while processing the request.
|
before or while processing the request.
|
||||||
key | value
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
3 | more data
|
|
||||||
3 | even more data
|
|
||||||
(2 rows)
|
|
||||||
|
|
||||||
COMMIT;
|
COMMIT;
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(2).kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(2).kill()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
|
|
|
@ -41,7 +41,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^ANALYZE").kill()');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
ANALYZE vacuum_test;
|
ANALYZE vacuum_test;
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: server closed the connection unexpectedly
|
||||||
This probably means the server terminated abnormally
|
This probably means the server terminated abnormally
|
||||||
before or while processing the request.
|
before or while processing the request.
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
||||||
|
@ -51,19 +51,38 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
ANALYZE vacuum_test;
|
ANALYZE vacuum_test;
|
||||||
-- ANALYZE transactions being critical is an open question, see #2430
|
WARNING: server closed the connection unexpectedly
|
||||||
-- show that we marked as INVALID on COMMIT FAILURE
|
This probably means the server terminated abnormally
|
||||||
SELECT shardid, shardstate FROM pg_dist_shard_placement where shardstate != 1 AND
|
before or while processing the request.
|
||||||
shardid in ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass);
|
connection not open
|
||||||
shardid | shardstate
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
|
WARNING: failed to commit transaction on localhost:xxxxx
|
||||||
|
WARNING: server closed the connection unexpectedly
|
||||||
|
This probably means the server terminated abnormally
|
||||||
|
before or while processing the request.
|
||||||
|
connection not open
|
||||||
|
connection not open
|
||||||
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
|
mitmproxy
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
12000000 | 3
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
SELECT recover_prepared_transactions();
|
||||||
WHERE shardid IN (
|
recover_prepared_transactions
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass
|
---------------------------------------------------------------------
|
||||||
);
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- ANALYZE transactions being critical is an open question, see #2430
|
||||||
|
-- show that we never mark as INVALID on COMMIT FAILURE
|
||||||
|
SELECT shardid, shardstate FROM pg_dist_shard_placement where shardstate != 1 AND
|
||||||
|
shardid in ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass);
|
||||||
|
shardid | shardstate
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
(0 rows)
|
||||||
|
|
||||||
-- the same tests with cancel
|
-- the same tests with cancel
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM").cancel(' || pg_backend_pid() || ')');
|
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM").cancel(' || pg_backend_pid() || ')');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
|
|
|
@ -1,4 +1,13 @@
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.shard_replication_factor TO 2;
|
||||||
|
CREATE TABLE the_replicated_table (a int, b int, z bigserial);
|
||||||
|
SELECT create_distributed_table('the_replicated_table', 'a');
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
CREATE TABLE the_table (a int, b int, z bigserial);
|
CREATE TABLE the_table (a int, b int, z bigserial);
|
||||||
SELECT create_distributed_table('the_table', 'a');
|
SELECT create_distributed_table('the_table', 'a');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
@ -39,10 +48,14 @@ INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
|
||||||
ERROR: writing to worker nodes is not currently allowed
|
ERROR: writing to worker nodes is not currently allowed
|
||||||
DETAIL: the database is read-only
|
DETAIL: the database is read-only
|
||||||
-- We can allow DML on a writable standby coordinator.
|
-- We can allow DML on a writable standby coordinator.
|
||||||
-- Note that it doesn't help to enable writes for citus local tables
|
-- Note that it doesn't help to enable writes for
|
||||||
-- and coordinator replicated reference tables. This is because, the
|
-- (a) citus local tables
|
||||||
-- data is in the coordinator and will hit read-only tranaction checks
|
-- (b) coordinator replicated reference tables.
|
||||||
-- on Postgres
|
-- (c) reference tables or replication > 1 distributed tables
|
||||||
|
-- (a) and (b) is because the data is in the coordinator and will hit
|
||||||
|
-- read-only tranaction checks on Postgres
|
||||||
|
-- (c) is because citus uses 2PC, where a transaction record should
|
||||||
|
-- be inserted to pg_dist_node, which is not allowed
|
||||||
SET citus.writable_standby_coordinator TO on;
|
SET citus.writable_standby_coordinator TO on;
|
||||||
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
|
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
|
||||||
SELECT * FROM the_table;
|
SELECT * FROM the_table;
|
||||||
|
@ -51,8 +64,19 @@ SELECT * FROM the_table;
|
||||||
1 | 2 | 2
|
1 | 2 | 2
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
INSERT INTO the_replicated_table (a, b, z) VALUES (1, 2, 2);
|
||||||
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
|
SELECT * FROM the_replicated_table;
|
||||||
|
a | b | z
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
(0 rows)
|
||||||
|
|
||||||
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
|
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
|
||||||
ERROR: cannot execute INSERT in a read-only transaction
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
SELECT * FROM reference_table;
|
SELECT * FROM reference_table;
|
||||||
a | b | z
|
a | b | z
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -66,8 +90,14 @@ SELECT * FROM citus_local_table;
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
UPDATE the_table SET z = 3 WHERE a = 1;
|
UPDATE the_table SET z = 3 WHERE a = 1;
|
||||||
|
UPDATE the_replicated_table SET z = 3 WHERE a = 1;
|
||||||
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
UPDATE reference_table SET z = 3 WHERE a = 1;
|
UPDATE reference_table SET z = 3 WHERE a = 1;
|
||||||
ERROR: cannot execute UPDATE in a read-only transaction
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
UPDATE citus_local_table SET z = 3 WHERE a = 1;
|
UPDATE citus_local_table SET z = 3 WHERE a = 1;
|
||||||
ERROR: cannot execute UPDATE in a read-only transaction
|
ERROR: cannot execute UPDATE in a read-only transaction
|
||||||
SELECT * FROM the_table;
|
SELECT * FROM the_table;
|
||||||
|
@ -87,8 +117,14 @@ SELECT * FROM citus_local_table;
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
DELETE FROM the_table WHERE a = 1;
|
DELETE FROM the_table WHERE a = 1;
|
||||||
|
DELETE FROM the_replicated_table WHERE a = 1;
|
||||||
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
DELETE FROM reference_table WHERE a = 1;
|
DELETE FROM reference_table WHERE a = 1;
|
||||||
ERROR: cannot execute DELETE in a read-only transaction
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
DELETE FROM citus_local_table WHERE a = 1;
|
DELETE FROM citus_local_table WHERE a = 1;
|
||||||
ERROR: cannot execute DELETE in a read-only transaction
|
ERROR: cannot execute DELETE in a read-only transaction
|
||||||
SELECT * FROM the_table;
|
SELECT * FROM the_table;
|
||||||
|
@ -109,20 +145,35 @@ SELECT * FROM citus_local_table;
|
||||||
-- drawing from a sequence is not possible
|
-- drawing from a sequence is not possible
|
||||||
INSERT INTO the_table (a, b) VALUES (1, 2);
|
INSERT INTO the_table (a, b) VALUES (1, 2);
|
||||||
ERROR: cannot assign TransactionIds during recovery
|
ERROR: cannot assign TransactionIds during recovery
|
||||||
|
INSERT INTO the_replicated_table (a, b) VALUES (1, 2);
|
||||||
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
INSERT INTO reference_table (a, b) VALUES (1, 2);
|
INSERT INTO reference_table (a, b) VALUES (1, 2);
|
||||||
ERROR: cannot assign TransactionIds during recovery
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
INSERT INTO citus_local_table (a, b) VALUES (1, 2);
|
INSERT INTO citus_local_table (a, b) VALUES (1, 2);
|
||||||
ERROR: cannot assign TransactionIds during recovery
|
ERROR: cannot assign TransactionIds during recovery
|
||||||
-- 2PC is not possible
|
-- 2PC is not possible
|
||||||
INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
ERROR: cannot assign TransactionIds during recovery
|
ERROR: cannot assign TransactionIds during recovery
|
||||||
|
INSERT INTO the_replicated_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
ERROR: cannot execute INSERT in a read-only transaction
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
INSERT INTO citus_local_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
INSERT INTO citus_local_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
ERROR: cannot execute INSERT in a read-only transaction
|
ERROR: cannot execute INSERT in a read-only transaction
|
||||||
-- COPY is not possible in 2PC mode
|
-- COPY is not possible in 2PC mode
|
||||||
COPY the_table (a, b, z) FROM STDIN WITH CSV;
|
COPY the_table (a, b, z) FROM STDIN WITH CSV;
|
||||||
ERROR: cannot assign TransactionIds during recovery
|
ERROR: cannot assign TransactionIds during recovery
|
||||||
|
-- COPY is not possible in 2PC mode
|
||||||
|
COPY the_replicated_table (a, b, z) FROM STDIN WITH CSV;
|
||||||
|
ERROR: cannot assign TransactionIds during recovery
|
||||||
COPY reference_table (a, b, z) FROM STDIN WITH CSV;
|
COPY reference_table (a, b, z) FROM STDIN WITH CSV;
|
||||||
ERROR: cannot assign TransactionIds during recovery
|
ERROR: cannot assign TransactionIds during recovery
|
||||||
COPY citus_local_table (a, b, z) FROM STDIN WITH CSV;
|
COPY citus_local_table (a, b, z) FROM STDIN WITH CSV;
|
||||||
|
@ -138,7 +189,9 @@ SELECT * FROM the_table ORDER BY a;
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
ERROR: cannot execute INSERT in a read-only transaction
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
SELECT * FROM reference_table ORDER BY a;
|
SELECT * FROM reference_table ORDER BY a;
|
||||||
a | b | z
|
a | b | z
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -162,7 +215,14 @@ SELECT * FROM del ORDER BY a;
|
||||||
|
|
||||||
WITH del AS (DELETE FROM reference_table RETURNING *)
|
WITH del AS (DELETE FROM reference_table RETURNING *)
|
||||||
SELECT * FROM del ORDER BY a;
|
SELECT * FROM del ORDER BY a;
|
||||||
ERROR: cannot execute DELETE in a read-only transaction
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
|
WITH del AS (DELETE FROM the_replicated_table RETURNING *)
|
||||||
|
SELECT * FROM del ORDER BY a;
|
||||||
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
WITH del AS (DELETE FROM citus_local_table RETURNING *)
|
WITH del AS (DELETE FROM citus_local_table RETURNING *)
|
||||||
SELECT * FROM del ORDER BY a;
|
SELECT * FROM del ORDER BY a;
|
||||||
ERROR: cannot execute DELETE in a read-only transaction
|
ERROR: cannot execute DELETE in a read-only transaction
|
||||||
|
@ -192,7 +252,9 @@ SELECT * FROM citus_local_table ORDER BY a;
|
||||||
|
|
||||||
DELETE FROM the_table;
|
DELETE FROM the_table;
|
||||||
DELETE FROM reference_table;
|
DELETE FROM reference_table;
|
||||||
ERROR: cannot execute DELETE in a read-only transaction
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
DELETE FROM citus_local_table;
|
DELETE FROM citus_local_table;
|
||||||
ERROR: cannot execute DELETE in a read-only transaction
|
ERROR: cannot execute DELETE in a read-only transaction
|
||||||
-- DDL is not possible
|
-- DDL is not possible
|
||||||
|
@ -217,7 +279,9 @@ INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
|
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
|
||||||
ERROR: cannot execute INSERT in a read-only transaction
|
ERROR: writing to worker nodes is not currently allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1.
|
||||||
|
DETAIL: the database is read-only
|
||||||
|
HINT: All modifications to replicated tables happen via 2PC, and 2PC requires the database to be in a writable state.
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
|
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
|
||||||
|
|
|
@ -369,7 +369,8 @@ ORDER BY nodeport, shardid;
|
||||||
|
|
||||||
-- hide postgresql version dependend messages for next test only
|
-- hide postgresql version dependend messages for next test only
|
||||||
\set VERBOSITY terse
|
\set VERBOSITY terse
|
||||||
-- deferred check should abort the transaction
|
-- for replicated tables use 2PC even if multi-shard commit protocol
|
||||||
|
-- is set to 2PC
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SET LOCAL citus.multi_shard_commit_protocol TO '1pc';
|
SET LOCAL citus.multi_shard_commit_protocol TO '1pc';
|
||||||
DELETE FROM researchers WHERE lab_id = 6;
|
DELETE FROM researchers WHERE lab_id = 6;
|
||||||
|
@ -377,11 +378,8 @@ DELETE FROM researchers WHERE lab_id = 6;
|
||||||
\copy researchers FROM STDIN delimiter ','
|
\copy researchers FROM STDIN delimiter ','
|
||||||
COMMIT;
|
COMMIT;
|
||||||
WARNING: illegal value
|
WARNING: illegal value
|
||||||
WARNING: failed to commit transaction on localhost:xxxxx
|
WARNING: connection to the remote node localhost:xxxxx failed with the following error: another command is already in progress
|
||||||
WARNING: illegal value
|
ERROR: illegal value
|
||||||
WARNING: failed to commit transaction on localhost:xxxxx
|
|
||||||
WARNING: could not commit transaction for shard xxxxx on any active node
|
|
||||||
ERROR: could not commit transaction on any active node
|
|
||||||
\unset VERBOSITY
|
\unset VERBOSITY
|
||||||
-- verify everyhing including delete is rolled back
|
-- verify everyhing including delete is rolled back
|
||||||
SELECT * FROM researchers WHERE lab_id = 6;
|
SELECT * FROM researchers WHERE lab_id = 6;
|
||||||
|
@ -614,21 +612,20 @@ DEFERRABLE INITIALLY DEFERRED
|
||||||
FOR EACH ROW EXECUTE PROCEDURE reject_bad();
|
FOR EACH ROW EXECUTE PROCEDURE reject_bad();
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
-- should be the same story as before, just at COMMIT time
|
-- should be the same story as before, just at COMMIT time
|
||||||
|
-- as we use 2PC, the transaction is rollbacked
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO objects VALUES (1, 'apple');
|
INSERT INTO objects VALUES (1, 'apple');
|
||||||
INSERT INTO objects VALUES (2, 'BAD');
|
INSERT INTO objects VALUES (2, 'BAD');
|
||||||
INSERT INTO labs VALUES (9, 'Umbrella Corporation');
|
INSERT INTO labs VALUES (9, 'Umbrella Corporation');
|
||||||
COMMIT;
|
COMMIT;
|
||||||
WARNING: illegal value
|
ERROR: illegal value
|
||||||
WARNING: failed to commit transaction on localhost:xxxxx
|
-- data should not persisted
|
||||||
-- data should be persisted
|
|
||||||
SELECT * FROM objects WHERE id = 2;
|
SELECT * FROM objects WHERE id = 2;
|
||||||
id | name
|
id | name
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
2 | BAD
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
-- but one placement should be bad
|
-- and nonne of the placements should be bad
|
||||||
SELECT count(*)
|
SELECT count(*)
|
||||||
FROM pg_dist_shard_placement AS sp,
|
FROM pg_dist_shard_placement AS sp,
|
||||||
pg_dist_shard AS s
|
pg_dist_shard AS s
|
||||||
|
@ -639,7 +636,7 @@ AND sp.shardstate = 3
|
||||||
AND s.logicalrelid = 'objects'::regclass;
|
AND s.logicalrelid = 'objects'::regclass;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DELETE FROM objects;
|
DELETE FROM objects;
|
||||||
|
@ -663,12 +660,8 @@ INSERT INTO labs VALUES (8, 'Aperture Science');
|
||||||
INSERT INTO labs VALUES (9, 'BAD');
|
INSERT INTO labs VALUES (9, 'BAD');
|
||||||
COMMIT;
|
COMMIT;
|
||||||
WARNING: illegal value
|
WARNING: illegal value
|
||||||
WARNING: failed to commit transaction on localhost:xxxxx
|
WARNING: connection to the remote node localhost:xxxxx failed with the following error: another command is already in progress
|
||||||
WARNING: illegal value
|
ERROR: illegal value
|
||||||
WARNING: failed to commit transaction on localhost:xxxxx
|
|
||||||
WARNING: could not commit transaction for shard xxxxx on any active node
|
|
||||||
WARNING: could not commit transaction for shard xxxxx on any active node
|
|
||||||
ERROR: could not commit transaction on any active node
|
|
||||||
-- data should NOT be persisted
|
-- data should NOT be persisted
|
||||||
SELECT * FROM objects WHERE id = 1;
|
SELECT * FROM objects WHERE id = 1;
|
||||||
id | name
|
id | name
|
||||||
|
@ -703,23 +696,20 @@ INSERT INTO objects VALUES (1, 'apple');
|
||||||
INSERT INTO labs VALUES (8, 'Aperture Science');
|
INSERT INTO labs VALUES (8, 'Aperture Science');
|
||||||
INSERT INTO labs VALUES (9, 'BAD');
|
INSERT INTO labs VALUES (9, 'BAD');
|
||||||
COMMIT;
|
COMMIT;
|
||||||
WARNING: illegal value
|
ERROR: illegal value
|
||||||
WARNING: failed to commit transaction on localhost:xxxxx
|
|
||||||
WARNING: could not commit transaction for shard xxxxx on any active node
|
|
||||||
\set VERBOSITY default
|
\set VERBOSITY default
|
||||||
-- data to objects should be persisted, but labs should not...
|
-- none of the changes should be persisted
|
||||||
SELECT * FROM objects WHERE id = 1;
|
SELECT * FROM objects WHERE id = 1;
|
||||||
id | name
|
id | name
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1 | apple
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT * FROM labs WHERE id = 8;
|
SELECT * FROM labs WHERE id = 8;
|
||||||
id | name
|
id | name
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
-- labs should be healthy, but one object placement shouldn't be
|
-- all placements should be healthy
|
||||||
SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*)
|
SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*)
|
||||||
FROM pg_dist_shard_placement AS sp,
|
FROM pg_dist_shard_placement AS sp,
|
||||||
pg_dist_shard AS s
|
pg_dist_shard AS s
|
||||||
|
@ -731,9 +721,8 @@ ORDER BY s.logicalrelid, sp.shardstate;
|
||||||
logicalrelid | shardstate | count
|
logicalrelid | shardstate | count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
labs | 1 | 1
|
labs | 1 | 1
|
||||||
objects | 1 | 1
|
objects | 1 | 2
|
||||||
objects | 3 | 1
|
(2 rows)
|
||||||
(3 rows)
|
|
||||||
|
|
||||||
-- some append-partitioned tests for good measure
|
-- some append-partitioned tests for good measure
|
||||||
CREATE TABLE append_researchers ( LIKE researchers );
|
CREATE TABLE append_researchers ( LIKE researchers );
|
||||||
|
|
|
@ -2448,28 +2448,14 @@ NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
|
||||||
HINT: Connect to worker nodes directly to manually create all necessary users and roles.
|
HINT: Connect to worker nodes directly to manually create all necessary users and roles.
|
||||||
GRANT INSERT ON ALL TABLES IN SCHEMA public TO router_user;
|
GRANT INSERT ON ALL TABLES IN SCHEMA public TO router_user;
|
||||||
\c - router_user - :master_port
|
\c - router_user - :master_port
|
||||||
-- first test that it is marked invalid inside a transaction block
|
|
||||||
-- we will fail to connect to worker 2, since the user does not exist
|
-- we will fail to connect to worker 2, since the user does not exist
|
||||||
|
-- still, we never mark placements inactive. Instead, fail the transaction
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO failure_test VALUES (1, 1);
|
INSERT INTO failure_test VALUES (1, 1);
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: FATAL: role "router_user" does not exist
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: FATAL: role "router_user" does not exist
|
||||||
SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement
|
|
||||||
WHERE shardid IN (
|
|
||||||
SELECT shardid FROM pg_dist_shard
|
|
||||||
WHERE logicalrelid = 'failure_test'::regclass
|
|
||||||
)
|
|
||||||
ORDER BY placementid;
|
|
||||||
shardid | shardstate | nodename | nodeport
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
840017 | 1 | localhost | 57637
|
|
||||||
840017 | 3 | localhost | 57638
|
|
||||||
840018 | 1 | localhost | 57638
|
|
||||||
840018 | 1 | localhost | 57637
|
|
||||||
(4 rows)
|
|
||||||
|
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
INSERT INTO failure_test VALUES (2, 1);
|
INSERT INTO failure_test VALUES (2, 1);
|
||||||
WARNING: connection to the remote node localhost:xxxxx failed with the following error: FATAL: role "router_user" does not exist
|
ERROR: connection to the remote node localhost:xxxxx failed with the following error: FATAL: role "router_user" does not exist
|
||||||
SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement
|
SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement
|
||||||
WHERE shardid IN (
|
WHERE shardid IN (
|
||||||
SELECT shardid FROM pg_dist_shard
|
SELECT shardid FROM pg_dist_shard
|
||||||
|
@ -2480,7 +2466,7 @@ SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
840017 | 1 | localhost | 57637
|
840017 | 1 | localhost | 57637
|
||||||
840017 | 1 | localhost | 57638
|
840017 | 1 | localhost | 57638
|
||||||
840018 | 3 | localhost | 57638
|
840018 | 1 | localhost | 57638
|
||||||
840018 | 1 | localhost | 57637
|
840018 | 1 | localhost | 57637
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
|
|
|
@ -136,12 +136,18 @@ SELECT recover_prepared_transactions();
|
||||||
0
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- plain INSERT does not use 2PC
|
-- plain INSERT uses 2PC
|
||||||
INSERT INTO test_recovery VALUES ('hello');
|
INSERT INTO test_recovery VALUES ('hello');
|
||||||
SELECT count(*) FROM pg_dist_transaction;
|
SELECT count(*) FROM pg_dist_transaction;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
0
|
2
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT recover_prepared_transactions();
|
||||||
|
recover_prepared_transactions
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- Aborted DDL commands should not write transaction recovery records
|
-- Aborted DDL commands should not write transaction recovery records
|
||||||
|
|
|
@ -196,36 +196,11 @@ SELECT master_create_worker_shards('second_dustbunnies', 1, 2);
|
||||||
|
|
||||||
-- run VACUUM and ANALYZE against the table on the master
|
-- run VACUUM and ANALYZE against the table on the master
|
||||||
\c - - :master_host :master_port
|
\c - - :master_host :master_port
|
||||||
SET citus.log_remote_commands TO ON;
|
|
||||||
VACUUM dustbunnies;
|
VACUUM dustbunnies;
|
||||||
NOTICE: issuing VACUUM public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
ANALYZE dustbunnies;
|
ANALYZE dustbunnies;
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing ANALYZE public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing ANALYZE public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing COMMIT
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing COMMIT
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
-- send a VACUUM FULL and a VACUUM ANALYZE
|
-- send a VACUUM FULL and a VACUUM ANALYZE
|
||||||
VACUUM (FULL) dustbunnies;
|
VACUUM (FULL) dustbunnies;
|
||||||
NOTICE: issuing VACUUM (FULL) public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (FULL) public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
VACUUM ANALYZE dustbunnies;
|
VACUUM ANALYZE dustbunnies;
|
||||||
NOTICE: issuing VACUUM (ANALYZE) public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (ANALYZE) public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
\c - - :public_worker_1_host :worker_1_port
|
\c - - :public_worker_1_host :worker_1_port
|
||||||
-- disable auto-VACUUM for next test
|
-- disable auto-VACUUM for next test
|
||||||
ALTER TABLE dustbunnies_990002 SET (autovacuum_enabled = false);
|
ALTER TABLE dustbunnies_990002 SET (autovacuum_enabled = false);
|
||||||
|
@ -233,25 +208,8 @@ SELECT relfrozenxid AS frozenxid FROM pg_class WHERE oid='dustbunnies_990002'::r
|
||||||
\gset
|
\gset
|
||||||
-- send a VACUUM FREEZE after adding a new row
|
-- send a VACUUM FREEZE after adding a new row
|
||||||
\c - - :master_host :master_port
|
\c - - :master_host :master_port
|
||||||
SET citus.log_remote_commands TO ON;
|
|
||||||
INSERT INTO dustbunnies VALUES (5, 'peter');
|
INSERT INTO dustbunnies VALUES (5, 'peter');
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing INSERT INTO public.dustbunnies_990002 (id, name) VALUES (5, 'peter'::text)
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing INSERT INTO public.dustbunnies_990002 (id, name) VALUES (5, 'peter'::text)
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing COMMIT
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing COMMIT
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
VACUUM (FREEZE) dustbunnies;
|
VACUUM (FREEZE) dustbunnies;
|
||||||
NOTICE: issuing VACUUM (FREEZE) public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (FREEZE) public.dustbunnies_990002
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
-- verify that relfrozenxid increased
|
-- verify that relfrozenxid increased
|
||||||
\c - - :public_worker_1_host :worker_1_port
|
\c - - :public_worker_1_host :worker_1_port
|
||||||
SELECT relfrozenxid::text::integer > :frozenxid AS frozen_performed FROM pg_class
|
SELECT relfrozenxid::text::integer > :frozenxid AS frozen_performed FROM pg_class
|
||||||
|
@ -273,33 +231,8 @@ WHERE tablename = 'dustbunnies_990002' ORDER BY attname;
|
||||||
|
|
||||||
-- add NULL values, then perform column-specific ANALYZE
|
-- add NULL values, then perform column-specific ANALYZE
|
||||||
\c - - :master_host :master_port
|
\c - - :master_host :master_port
|
||||||
SET citus.log_remote_commands TO ON;
|
|
||||||
INSERT INTO dustbunnies VALUES (6, NULL, NULL);
|
INSERT INTO dustbunnies VALUES (6, NULL, NULL);
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing INSERT INTO public.dustbunnies_990002 (id, name, age) VALUES (6, NULL::text, NULL::integer)
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing INSERT INTO public.dustbunnies_990002 (id, name, age) VALUES (6, NULL::text, NULL::integer)
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing COMMIT
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing COMMIT
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
ANALYZE dustbunnies (name);
|
ANALYZE dustbunnies (name);
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing ANALYZE public.dustbunnies_990002 (name)
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing ANALYZE public.dustbunnies_990002 (name)
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing COMMIT
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing COMMIT
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
-- verify that name's NULL ratio is updated but age's is not
|
-- verify that name's NULL ratio is updated but age's is not
|
||||||
\c - - :public_worker_1_host :worker_1_port
|
\c - - :public_worker_1_host :worker_1_port
|
||||||
SELECT attname, null_frac FROM pg_stats
|
SELECT attname, null_frac FROM pg_stats
|
||||||
|
|
|
@ -7,6 +7,7 @@ SELECT substring(:'server_version', '\d+')::int > 13 AS server_version_above_thi
|
||||||
\endif
|
\endif
|
||||||
create schema pg14;
|
create schema pg14;
|
||||||
set search_path to pg14;
|
set search_path to pg14;
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
SET citus.next_shard_id TO 980000;
|
SET citus.next_shard_id TO 980000;
|
||||||
SET citus.shard_count TO 2;
|
SET citus.shard_count TO 2;
|
||||||
-- test the new vacuum option, process_toast
|
-- test the new vacuum option, process_toast
|
||||||
|
@ -21,28 +22,16 @@ SET citus.log_remote_commands TO ON;
|
||||||
VACUUM (FULL) t1;
|
VACUUM (FULL) t1;
|
||||||
NOTICE: issuing VACUUM (FULL) pg14.t1_980000
|
NOTICE: issuing VACUUM (FULL) pg14.t1_980000
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing VACUUM (FULL) pg14.t1_980000
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (FULL) pg14.t1_980001
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (FULL) pg14.t1_980001
|
NOTICE: issuing VACUUM (FULL) pg14.t1_980001
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
VACUUM (FULL, PROCESS_TOAST) t1;
|
VACUUM (FULL, PROCESS_TOAST) t1;
|
||||||
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980000
|
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980000
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980000
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980001
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980001
|
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980001
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
VACUUM (FULL, PROCESS_TOAST true) t1;
|
VACUUM (FULL, PROCESS_TOAST true) t1;
|
||||||
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980000
|
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980000
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980000
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980001
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980001
|
NOTICE: issuing VACUUM (FULL,PROCESS_TOAST) pg14.t1_980001
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
VACUUM (FULL, PROCESS_TOAST false) t1;
|
VACUUM (FULL, PROCESS_TOAST false) t1;
|
||||||
|
@ -50,10 +39,6 @@ ERROR: PROCESS_TOAST required with VACUUM FULL
|
||||||
VACUUM (PROCESS_TOAST false) t1;
|
VACUUM (PROCESS_TOAST false) t1;
|
||||||
NOTICE: issuing VACUUM pg14.t1_980000
|
NOTICE: issuing VACUUM pg14.t1_980000
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing VACUUM pg14.t1_980000
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM pg14.t1_980001
|
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
|
||||||
NOTICE: issuing VACUUM pg14.t1_980001
|
NOTICE: issuing VACUUM pg14.t1_980001
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
SET citus.log_remote_commands TO OFF;
|
SET citus.log_remote_commands TO OFF;
|
||||||
|
@ -74,17 +59,29 @@ NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET search_path TO pg14;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET search_path TO pg14;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing reindex(TABLESPACE test_tablespace) index idx;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing reindex(TABLESPACE test_tablespace) index idx;
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing PREPARE TRANSACTION 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing COMMIT
|
NOTICE: issuing PREPARE TRANSACTION 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing COMMIT
|
NOTICE: issuing COMMIT PREPARED 'citus_xx_xx_xx_xx'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing COMMIT PREPARED 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
reindex(TABLESPACE test_tablespace, verbose) index idx;
|
reindex(TABLESPACE test_tablespace, verbose) index idx;
|
||||||
INFO: index "idx" was reindexed
|
INFO: index "idx" was reindexed
|
||||||
|
@ -92,34 +89,58 @@ NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET search_path TO pg14;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET search_path TO pg14;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing reindex(TABLESPACE test_tablespace, verbose) index idx;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing reindex(TABLESPACE test_tablespace, verbose) index idx;
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing PREPARE TRANSACTION 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing COMMIT
|
NOTICE: issuing PREPARE TRANSACTION 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing COMMIT
|
NOTICE: issuing COMMIT PREPARED 'citus_xx_xx_xx_xx'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing COMMIT PREPARED 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
reindex(TABLESPACE test_tablespace, verbose false) index idx ;
|
reindex(TABLESPACE test_tablespace, verbose false) index idx ;
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET search_path TO pg14;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET search_path TO pg14;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing reindex(TABLESPACE test_tablespace, verbose false) index idx ;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing reindex(TABLESPACE test_tablespace, verbose false) index idx ;
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing PREPARE TRANSACTION 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing COMMIT
|
NOTICE: issuing PREPARE TRANSACTION 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing COMMIT
|
NOTICE: issuing COMMIT PREPARED 'citus_xx_xx_xx_xx'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing COMMIT PREPARED 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
reindex(verbose, TABLESPACE test_tablespace) index idx ;
|
reindex(verbose, TABLESPACE test_tablespace) index idx ;
|
||||||
INFO: index "idx" was reindexed
|
INFO: index "idx" was reindexed
|
||||||
|
@ -127,17 +148,29 @@ NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET search_path TO pg14;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing SET search_path TO pg14;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing reindex(verbose, TABLESPACE test_tablespace) index idx ;
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing reindex(verbose, TABLESPACE test_tablespace) index idx ;
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing REINDEX (VERBOSE, TABLESPACE test_tablespace) INDEX pg14.xxxxx
|
NOTICE: issuing PREPARE TRANSACTION 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing COMMIT
|
NOTICE: issuing PREPARE TRANSACTION 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
NOTICE: issuing COMMIT
|
NOTICE: issuing COMMIT PREPARED 'citus_xx_xx_xx_xx'
|
||||||
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
|
NOTICE: issuing COMMIT PREPARED 'citus_xx_xx_xx_xx'
|
||||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
-- should error saying table space doesn't exist
|
-- should error saying table space doesn't exist
|
||||||
reindex(TABLESPACE test_tablespace1) index idx;
|
reindex(TABLESPACE test_tablespace1) index idx;
|
||||||
|
|
|
@ -115,7 +115,8 @@ SELECT distributed_2PCs_are_equal_to_placement_count();
|
||||||
t
|
t
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- with 1PC, we should not see and distributed TXs in the pg_dist_transaction
|
-- even if 1PC used, we use 2PC as we modify replicated tables
|
||||||
|
-- see distributed TXs in the pg_dist_transaction
|
||||||
SET citus.multi_shard_commit_protocol TO '1pc';
|
SET citus.multi_shard_commit_protocol TO '1pc';
|
||||||
SET citus.multi_shard_modify_mode TO 'sequential';
|
SET citus.multi_shard_modify_mode TO 'sequential';
|
||||||
SELECT recover_prepared_transactions();
|
SELECT recover_prepared_transactions();
|
||||||
|
@ -128,7 +129,7 @@ ALTER TABLE test_table ADD CONSTRAINT c_check CHECK(a > 0);
|
||||||
SELECT no_distributed_2PCs();
|
SELECT no_distributed_2PCs();
|
||||||
no_distributed_2pcs
|
no_distributed_2pcs
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
t
|
f
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SET citus.multi_shard_commit_protocol TO '1pc';
|
SET citus.multi_shard_commit_protocol TO '1pc';
|
||||||
|
@ -143,7 +144,7 @@ ALTER TABLE test_table ADD CONSTRAINT d_check CHECK(a > 0);
|
||||||
SELECT no_distributed_2PCs();
|
SELECT no_distributed_2PCs();
|
||||||
no_distributed_2pcs
|
no_distributed_2pcs
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
t
|
f
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
CREATE TABLE ref_test(a int);
|
CREATE TABLE ref_test(a int);
|
||||||
|
@ -194,7 +195,7 @@ SELECT create_distributed_table('test_table_rep_2', 'a');
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- 1PC should never use 2PC with rep > 1
|
-- even if 1PC used, we use 2PC with rep > 1
|
||||||
SET citus.multi_shard_commit_protocol TO '1pc';
|
SET citus.multi_shard_commit_protocol TO '1pc';
|
||||||
SET citus.multi_shard_modify_mode TO 'sequential';
|
SET citus.multi_shard_modify_mode TO 'sequential';
|
||||||
SELECT recover_prepared_transactions();
|
SELECT recover_prepared_transactions();
|
||||||
|
@ -207,7 +208,7 @@ CREATE INDEX test_table_rep_2_i_1 ON test_table_rep_2(a);
|
||||||
SELECT no_distributed_2PCs();
|
SELECT no_distributed_2PCs();
|
||||||
no_distributed_2pcs
|
no_distributed_2pcs
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
t
|
f
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SET citus.multi_shard_modify_mode TO 'parallel';
|
SET citus.multi_shard_modify_mode TO 'parallel';
|
||||||
|
@ -221,7 +222,7 @@ CREATE INDEX test_table_rep_2_i_2 ON test_table_rep_2(a);
|
||||||
SELECT no_distributed_2PCs();
|
SELECT no_distributed_2PCs();
|
||||||
no_distributed_2pcs
|
no_distributed_2pcs
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
t
|
f
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- 2PC should always use 2PC with rep > 1
|
-- 2PC should always use 2PC with rep > 1
|
||||||
|
|
|
@ -308,14 +308,15 @@ COMMIT;
|
||||||
-- Nothing from the block should have committed
|
-- Nothing from the block should have committed
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1;
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1;
|
||||||
|
|
||||||
-- Now try with 2pc off
|
-- Even if 1PC is picked for multi-shard commands
|
||||||
|
-- Citus always uses 2PC for replication > 1
|
||||||
SET citus.multi_shard_commit_protocol TO '1pc';
|
SET citus.multi_shard_commit_protocol TO '1pc';
|
||||||
BEGIN;
|
BEGIN;
|
||||||
CREATE INDEX single_index_2 ON single_shard_items(id);
|
CREATE INDEX single_index_2 ON single_shard_items(id);
|
||||||
CREATE INDEX single_index_3 ON single_shard_items(name);
|
CREATE INDEX single_index_3 ON single_shard_items(name);
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
-- The block should have committed with a warning
|
-- Nothing from the block should have committed
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1;
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1;
|
||||||
|
|
||||||
\c - - - :worker_2_port
|
\c - - - :worker_2_port
|
||||||
|
|
|
@ -26,7 +26,7 @@ CREATE TABLE lineitem_alter (
|
||||||
WITH ( fillfactor = 80 );
|
WITH ( fillfactor = 80 );
|
||||||
SELECT create_distributed_table('lineitem_alter', 'l_orderkey', 'append');
|
SELECT create_distributed_table('lineitem_alter', 'l_orderkey', 'append');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -34,14 +34,14 @@ SELECT create_distributed_table('lineitem_alter', 'l_orderkey', 'append');
|
||||||
-- verify that the storage options made it to the table definitions
|
-- verify that the storage options made it to the table definitions
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname = 'lineitem_alter';
|
SELECT relname, reloptions FROM pg_class WHERE relname = 'lineitem_alter';
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
----------------+-----------------
|
---------------------------------------------------------------------
|
||||||
lineitem_alter | {fillfactor=80}
|
lineitem_alter | {fillfactor=80}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'lineitem_alter%' ORDER BY relname;
|
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'lineitem_alter%' ORDER BY relname;
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
-----------------------+-----------------
|
---------------------------------------------------------------------
|
||||||
lineitem_alter_220000 | {fillfactor=80}
|
lineitem_alter_220000 | {fillfactor=80}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -60,7 +60,7 @@ FROM
|
||||||
JOIN pg_attribute ON (pc.oid = pg_attribute.attrelid)
|
JOIN pg_attribute ON (pc.oid = pg_attribute.attrelid)
|
||||||
ORDER BY attnum;
|
ORDER BY attnum;
|
||||||
attname | atttypid
|
attname | atttypid
|
||||||
-----------------+-------------------
|
---------------------------------------------------------------------
|
||||||
tableoid | oid
|
tableoid | oid
|
||||||
cmax | cid
|
cmax | cid
|
||||||
xmax | xid
|
xmax | xid
|
||||||
|
@ -93,7 +93,7 @@ ORDER BY attnum;
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+-----------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -119,13 +119,13 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineite
|
||||||
|
|
||||||
SELECT float_column, count(*) FROM lineitem_alter GROUP BY float_column;
|
SELECT float_column, count(*) FROM lineitem_alter GROUP BY float_column;
|
||||||
float_column | count
|
float_column | count
|
||||||
--------------+-------
|
---------------------------------------------------------------------
|
||||||
| 6000
|
| 6000
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT int_column1, count(*) FROM lineitem_alter GROUP BY int_column1;
|
SELECT int_column1, count(*) FROM lineitem_alter GROUP BY int_column1;
|
||||||
int_column1 | count
|
int_column1 | count
|
||||||
-------------+-------
|
---------------------------------------------------------------------
|
||||||
1 | 6000
|
1 | 6000
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -136,14 +136,14 @@ ALTER TABLE lineitem_alter ALTER COLUMN int_column1 DROP DEFAULT;
|
||||||
\copy lineitem_alter (l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment) FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|'
|
\copy lineitem_alter (l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment) FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|'
|
||||||
SELECT float_column, count(*) FROM lineitem_alter GROUP BY float_column;
|
SELECT float_column, count(*) FROM lineitem_alter GROUP BY float_column;
|
||||||
float_column | count
|
float_column | count
|
||||||
--------------+-------
|
---------------------------------------------------------------------
|
||||||
| 6000
|
| 6000
|
||||||
1 | 6000
|
1 | 6000
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
SELECT int_column1, count(*) FROM lineitem_alter GROUP BY int_column1;
|
SELECT int_column1, count(*) FROM lineitem_alter GROUP BY int_column1;
|
||||||
int_column1 | count
|
int_column1 | count
|
||||||
-------------+-------
|
---------------------------------------------------------------------
|
||||||
| 6000
|
| 6000
|
||||||
1 | 6000
|
1 | 6000
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
@ -152,7 +152,7 @@ SELECT int_column1, count(*) FROM lineitem_alter GROUP BY int_column1;
|
||||||
ALTER TABLE lineitem_alter ALTER COLUMN int_column2 SET NOT NULL;
|
ALTER TABLE lineitem_alter ALTER COLUMN int_column2 SET NOT NULL;
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+--------------------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -187,7 +187,7 @@ DETAIL: Failing row contains (1, 155190, 7706, 1, 17.00, 21168.23, 0.04, 0.02,
|
||||||
ALTER TABLE lineitem_alter ALTER COLUMN int_column2 DROP NOT NULL;
|
ALTER TABLE lineitem_alter ALTER COLUMN int_column2 DROP NOT NULL;
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+-----------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -215,14 +215,14 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineite
|
||||||
\copy lineitem_alter (l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment) FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|'
|
\copy lineitem_alter (l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment) FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|'
|
||||||
SELECT count(*) from lineitem_alter;
|
SELECT count(*) from lineitem_alter;
|
||||||
count
|
count
|
||||||
-------
|
---------------------------------------------------------------------
|
||||||
18000
|
18000
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- Verify that SET DATA TYPE works
|
-- Verify that SET DATA TYPE works
|
||||||
SELECT int_column2, pg_typeof(int_column2), count(*) from lineitem_alter GROUP BY int_column2;
|
SELECT int_column2, pg_typeof(int_column2), count(*) from lineitem_alter GROUP BY int_column2;
|
||||||
int_column2 | pg_typeof | count
|
int_column2 | pg_typeof | count
|
||||||
-------------+-----------+-------
|
---------------------------------------------------------------------
|
||||||
| integer | 6000
|
| integer | 6000
|
||||||
2 | integer | 12000
|
2 | integer | 12000
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
@ -230,7 +230,7 @@ SELECT int_column2, pg_typeof(int_column2), count(*) from lineitem_alter GROUP B
|
||||||
ALTER TABLE lineitem_alter ALTER COLUMN int_column2 SET DATA TYPE FLOAT;
|
ALTER TABLE lineitem_alter ALTER COLUMN int_column2 SET DATA TYPE FLOAT;
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+-----------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -256,7 +256,7 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineite
|
||||||
|
|
||||||
SELECT int_column2, pg_typeof(int_column2), count(*) from lineitem_alter GROUP BY int_column2;
|
SELECT int_column2, pg_typeof(int_column2), count(*) from lineitem_alter GROUP BY int_column2;
|
||||||
int_column2 | pg_typeof | count
|
int_column2 | pg_typeof | count
|
||||||
-------------+------------------+-------
|
---------------------------------------------------------------------
|
||||||
| double precision | 6000
|
| double precision | 6000
|
||||||
2 | double precision | 12000
|
2 | double precision | 12000
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
@ -269,7 +269,7 @@ ALTER TABLE lineitem_alter DROP COLUMN date_column;
|
||||||
ALTER TABLE lineitem_alter RENAME COLUMN l_orderkey TO l_orderkey_renamed;
|
ALTER TABLE lineitem_alter RENAME COLUMN l_orderkey TO l_orderkey_renamed;
|
||||||
SELECT SUM(l_orderkey_renamed) FROM lineitem_alter;
|
SELECT SUM(l_orderkey_renamed) FROM lineitem_alter;
|
||||||
sum
|
sum
|
||||||
----------
|
---------------------------------------------------------------------
|
||||||
53620791
|
53620791
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -288,13 +288,13 @@ ALTER TABLE lineitem_alter DROP COLUMN IF EXISTS int_column2;
|
||||||
ALTER TABLE IF EXISTS lineitem_alter RENAME COLUMN l_orderkey_renamed TO l_orderkey;
|
ALTER TABLE IF EXISTS lineitem_alter RENAME COLUMN l_orderkey_renamed TO l_orderkey;
|
||||||
SELECT SUM(l_orderkey) FROM lineitem_alter;
|
SELECT SUM(l_orderkey) FROM lineitem_alter;
|
||||||
sum
|
sum
|
||||||
----------
|
---------------------------------------------------------------------
|
||||||
53620791
|
53620791
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+-----------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -319,7 +319,7 @@ ALTER TABLE lineitem_alter ADD COLUMN int_column1 INTEGER,
|
||||||
ADD COLUMN int_column2 INTEGER;
|
ADD COLUMN int_column2 INTEGER;
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+-----------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -348,7 +348,7 @@ DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP|VAL
|
||||||
ALTER TABLE lineitem_alter DROP COLUMN int_column1, DROP COLUMN int_column2;
|
ALTER TABLE lineitem_alter DROP COLUMN int_column1, DROP COLUMN int_column2;
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+-----------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -386,11 +386,9 @@ DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT, ADD|DROP|VAL
|
||||||
-- types
|
-- types
|
||||||
ALTER TABLE lineitem_alter ADD COLUMN new_column non_existent_type;
|
ALTER TABLE lineitem_alter ADD COLUMN new_column non_existent_type;
|
||||||
ERROR: type "non_existent_type" does not exist
|
ERROR: type "non_existent_type" does not exist
|
||||||
LINE 1: ALTER TABLE lineitem_alter ADD COLUMN new_column non_existen...
|
|
||||||
^
|
|
||||||
ALTER TABLE lineitem_alter ALTER COLUMN null_column SET NOT NULL;
|
ALTER TABLE lineitem_alter ALTER COLUMN null_column SET NOT NULL;
|
||||||
ERROR: column "null_column" contains null values
|
ERROR: column "null_column" contains null values
|
||||||
CONTEXT: while executing command on localhost:57637
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
ALTER TABLE lineitem_alter ALTER COLUMN l_partkey SET DEFAULT 'a';
|
ALTER TABLE lineitem_alter ALTER COLUMN l_partkey SET DEFAULT 'a';
|
||||||
ERROR: invalid input syntax for integer: "a"
|
ERROR: invalid input syntax for integer: "a"
|
||||||
-- Verify that we error out on RENAME CONSTRAINT statement
|
-- Verify that we error out on RENAME CONSTRAINT statement
|
||||||
|
@ -407,7 +405,7 @@ NOTICE: relation "non_existent_table" does not exist, skipping
|
||||||
-- node
|
-- node
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+-----------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -434,7 +432,7 @@ CREATE INDEX temp_index_1 ON lineitem_alter(l_linenumber);
|
||||||
COMMIT;
|
COMMIT;
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
--------------+----------------
|
---------------------------------------------------------------------
|
||||||
temp_index_1 | lineitem_alter
|
temp_index_1 | lineitem_alter
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -446,7 +444,7 @@ CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
||||||
COMMIT;
|
COMMIT;
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
--------------+----------------
|
---------------------------------------------------------------------
|
||||||
temp_index_2 | lineitem_alter
|
temp_index_2 | lineitem_alter
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -458,7 +456,7 @@ ALTER TABLE lineitem_alter ADD COLUMN first integer;
|
||||||
COMMIT;
|
COMMIT;
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineitem_alter'::regclass;
|
||||||
Column | Type | Modifiers
|
Column | Type | Modifiers
|
||||||
-----------------+-----------------------+-----------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | not null
|
l_orderkey | bigint | not null
|
||||||
l_partkey | integer | not null
|
l_partkey | integer | not null
|
||||||
l_suppkey | integer | not null
|
l_suppkey | integer | not null
|
||||||
|
@ -482,7 +480,7 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.lineite
|
||||||
SELECT "Column", "Type", "Definition" FROM index_attrs WHERE
|
SELECT "Column", "Type", "Definition" FROM index_attrs WHERE
|
||||||
relid = 'temp_index_2'::regclass;
|
relid = 'temp_index_2'::regclass;
|
||||||
Column | Type | Definition
|
Column | Type | Definition
|
||||||
------------+--------+------------
|
---------------------------------------------------------------------
|
||||||
l_orderkey | bigint | l_orderkey
|
l_orderkey | bigint | l_orderkey
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -495,7 +493,7 @@ CREATE INDEX temp_index_3 ON lineitem_alter(l_partkey);
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
-----------+-----------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
-- ensure that errors cause full rollback
|
-- ensure that errors cause full rollback
|
||||||
|
@ -506,7 +504,7 @@ ERROR: relation "temp_index_2" already exists
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
-----------+-----------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
-- verify that SAVEPOINT is allowed...
|
-- verify that SAVEPOINT is allowed...
|
||||||
|
@ -524,7 +522,7 @@ ROLLBACK TO my_savepoint;
|
||||||
COMMIT;
|
COMMIT;
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
--------------+----------------
|
---------------------------------------------------------------------
|
||||||
temp_index_2 | lineitem_alter
|
temp_index_2 | lineitem_alter
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -538,12 +536,12 @@ BEGIN;
|
||||||
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
||||||
ALTER TABLE lineitem_alter ADD COLUMN first integer;
|
ALTER TABLE lineitem_alter ADD COLUMN first integer;
|
||||||
ERROR: column "first" of relation "lineitem_alter_220000" already exists
|
ERROR: column "first" of relation "lineitem_alter_220000" already exists
|
||||||
CONTEXT: while executing command on localhost:57638
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
COMMIT;
|
COMMIT;
|
||||||
-- Nothing from the block should have committed
|
-- Nothing from the block should have committed
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
-----------+-----------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
-- Create single-shard table (to avoid deadlocks in the upcoming test hackery)
|
-- Create single-shard table (to avoid deadlocks in the upcoming test hackery)
|
||||||
|
@ -552,7 +550,7 @@ SET citus.shard_count TO 1;
|
||||||
SET citus.shard_replication_factor TO 2;
|
SET citus.shard_replication_factor TO 2;
|
||||||
SELECT create_distributed_table('single_shard_items', 'id', 'hash');
|
SELECT create_distributed_table('single_shard_items', 'id', 'hash');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -560,13 +558,13 @@ SELECT create_distributed_table('single_shard_items', 'id', 'hash');
|
||||||
CREATE UNIQUE INDEX replica_idx on single_shard_items(id);
|
CREATE UNIQUE INDEX replica_idx on single_shard_items(id);
|
||||||
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
||||||
relreplident
|
relreplident
|
||||||
--------------
|
---------------------------------------------------------------------
|
||||||
d
|
d
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
||||||
run_command_on_workers
|
run_command_on_workers
|
||||||
------------------------
|
---------------------------------------------------------------------
|
||||||
(localhost,57637,t,d)
|
(localhost,57637,t,d)
|
||||||
(localhost,57638,t,d)
|
(localhost,57638,t,d)
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
@ -574,13 +572,13 @@ SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname L
|
||||||
ALTER TABLE single_shard_items REPLICA IDENTITY nothing;
|
ALTER TABLE single_shard_items REPLICA IDENTITY nothing;
|
||||||
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
||||||
relreplident
|
relreplident
|
||||||
--------------
|
---------------------------------------------------------------------
|
||||||
n
|
n
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
||||||
run_command_on_workers
|
run_command_on_workers
|
||||||
------------------------
|
---------------------------------------------------------------------
|
||||||
(localhost,57637,t,n)
|
(localhost,57637,t,n)
|
||||||
(localhost,57638,t,n)
|
(localhost,57638,t,n)
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
@ -588,13 +586,13 @@ SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname L
|
||||||
ALTER TABLE single_shard_items REPLICA IDENTITY full;
|
ALTER TABLE single_shard_items REPLICA IDENTITY full;
|
||||||
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
||||||
relreplident
|
relreplident
|
||||||
--------------
|
---------------------------------------------------------------------
|
||||||
f
|
f
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
||||||
run_command_on_workers
|
run_command_on_workers
|
||||||
------------------------
|
---------------------------------------------------------------------
|
||||||
(localhost,57637,t,f)
|
(localhost,57637,t,f)
|
||||||
(localhost,57638,t,f)
|
(localhost,57638,t,f)
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
@ -602,13 +600,13 @@ SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname L
|
||||||
ALTER TABLE single_shard_items REPLICA IDENTITY USING INDEX replica_idx;
|
ALTER TABLE single_shard_items REPLICA IDENTITY USING INDEX replica_idx;
|
||||||
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
||||||
relreplident
|
relreplident
|
||||||
--------------
|
---------------------------------------------------------------------
|
||||||
i
|
i
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
||||||
run_command_on_workers
|
run_command_on_workers
|
||||||
------------------------
|
---------------------------------------------------------------------
|
||||||
(localhost,57637,t,i)
|
(localhost,57637,t,i)
|
||||||
(localhost,57638,t,i)
|
(localhost,57638,t,i)
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
@ -616,13 +614,13 @@ SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname L
|
||||||
ALTER TABLE single_shard_items REPLICA IDENTITY default, REPLICA IDENTITY USING INDEX replica_idx, REPLICA IDENTITY nothing;
|
ALTER TABLE single_shard_items REPLICA IDENTITY default, REPLICA IDENTITY USING INDEX replica_idx, REPLICA IDENTITY nothing;
|
||||||
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
SELECT relreplident FROM pg_class WHERE relname = 'single_shard_items';
|
||||||
relreplident
|
relreplident
|
||||||
--------------
|
---------------------------------------------------------------------
|
||||||
n
|
n
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
SELECT run_command_on_workers('SELECT relreplident FROM pg_class WHERE relname LIKE ''single_shard_items_%'' LIMIT 1;');
|
||||||
run_command_on_workers
|
run_command_on_workers
|
||||||
------------------------
|
---------------------------------------------------------------------
|
||||||
(localhost,57637,t,n)
|
(localhost,57637,t,n)
|
||||||
(localhost,57638,t,n)
|
(localhost,57638,t,n)
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
@ -652,30 +650,28 @@ CREATE INDEX single_index_3 ON single_shard_items(name);
|
||||||
COMMIT;
|
COMMIT;
|
||||||
ERROR: duplicate key value violates unique constraint "ddl_commands_command_key"
|
ERROR: duplicate key value violates unique constraint "ddl_commands_command_key"
|
||||||
DETAIL: Key (command)=(CREATE INDEX) already exists.
|
DETAIL: Key (command)=(CREATE INDEX) already exists.
|
||||||
CONTEXT: while executing command on localhost:57638
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
-- Nothing from the block should have committed
|
-- Nothing from the block should have committed
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1;
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1;
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
-----------+-----------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
-- Now try with 2pc off
|
-- Even if 1PC is picked for multi-shard commands
|
||||||
|
-- Citus always uses 2PC for replication > 1
|
||||||
SET citus.multi_shard_commit_protocol TO '1pc';
|
SET citus.multi_shard_commit_protocol TO '1pc';
|
||||||
BEGIN;
|
BEGIN;
|
||||||
CREATE INDEX single_index_2 ON single_shard_items(id);
|
CREATE INDEX single_index_2 ON single_shard_items(id);
|
||||||
CREATE INDEX single_index_3 ON single_shard_items(name);
|
CREATE INDEX single_index_3 ON single_shard_items(name);
|
||||||
COMMIT;
|
COMMIT;
|
||||||
WARNING: duplicate key value violates unique constraint "ddl_commands_command_key"
|
ERROR: duplicate key value violates unique constraint "ddl_commands_command_key"
|
||||||
DETAIL: Key (command)=(CREATE INDEX) already exists.
|
DETAIL: Key (command)=(CREATE INDEX) already exists.
|
||||||
CONTEXT: while executing command on localhost:57638
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
WARNING: failed to commit transaction on localhost:57638
|
-- Nothing from the block should have committed
|
||||||
-- The block should have committed with a warning
|
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1;
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1;
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
----------------+--------------------
|
---------------------------------------------------------------------
|
||||||
single_index_2 | single_shard_items
|
(0 rows)
|
||||||
single_index_3 | single_shard_items
|
|
||||||
(2 rows)
|
|
||||||
|
|
||||||
\c - - - :worker_2_port
|
\c - - - :worker_2_port
|
||||||
DROP EVENT TRIGGER log_ddl_tag;
|
DROP EVENT TRIGGER log_ddl_tag;
|
||||||
|
@ -687,7 +683,7 @@ BEGIN;
|
||||||
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
||||||
SELECT count(*) FROM lineitem_alter;
|
SELECT count(*) FROM lineitem_alter;
|
||||||
count
|
count
|
||||||
-------
|
---------------------------------------------------------------------
|
||||||
18000
|
18000
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -696,7 +692,7 @@ ROLLBACK;
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SELECT count(*) FROM lineitem_alter;
|
SELECT count(*) FROM lineitem_alter;
|
||||||
count
|
count
|
||||||
-------
|
---------------------------------------------------------------------
|
||||||
18000
|
18000
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -704,7 +700,7 @@ CREATE INDEX temp_index_2 ON lineitem_alter(l_orderkey);
|
||||||
COMMIT;
|
COMMIT;
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
--------------+----------------
|
---------------------------------------------------------------------
|
||||||
temp_index_2 | lineitem_alter
|
temp_index_2 | lineitem_alter
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -714,14 +710,14 @@ SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
CREATE INDEX temp_index_3 ON lineitem_alter(l_orderkey);
|
CREATE INDEX temp_index_3 ON lineitem_alter(l_orderkey);
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
--------------+----------------
|
---------------------------------------------------------------------
|
||||||
temp_index_3 | lineitem_alter
|
temp_index_3 | lineitem_alter
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DROP INDEX temp_index_3;
|
DROP INDEX temp_index_3;
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
-----------+-----------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
RESET citus.multi_shard_commit_protocol;
|
RESET citus.multi_shard_commit_protocol;
|
||||||
|
@ -730,7 +726,7 @@ CREATE TABLE test_ab (a int, b int);
|
||||||
SET citus.shard_count TO 8;
|
SET citus.shard_count TO 8;
|
||||||
SELECT create_distributed_table('test_ab', 'a', 'hash');
|
SELECT create_distributed_table('test_ab', 'a', 'hash');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -739,11 +735,11 @@ INSERT INTO test_ab VALUES (2, 11);
|
||||||
CREATE UNIQUE INDEX temp_unique_index_1 ON test_ab(a);
|
CREATE UNIQUE INDEX temp_unique_index_1 ON test_ab(a);
|
||||||
ERROR: could not create unique index "temp_unique_index_1_220011"
|
ERROR: could not create unique index "temp_unique_index_1_220011"
|
||||||
DETAIL: Key (a)=(2) is duplicated.
|
DETAIL: Key (a)=(2) is duplicated.
|
||||||
CONTEXT: while executing command on localhost:57638
|
CONTEXT: while executing command on localhost:xxxxx
|
||||||
SELECT shardid FROM pg_dist_shard_placement NATURAL JOIN pg_dist_shard
|
SELECT shardid FROM pg_dist_shard_placement NATURAL JOIN pg_dist_shard
|
||||||
WHERE logicalrelid='test_ab'::regclass AND shardstate=3;
|
WHERE logicalrelid='test_ab'::regclass AND shardstate=3;
|
||||||
shardid
|
shardid
|
||||||
---------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
-- Check that the schema on the worker still looks reasonable
|
-- Check that the schema on the worker still looks reasonable
|
||||||
|
@ -754,7 +750,7 @@ FROM
|
||||||
JOIN pg_attribute ON (pc.oid = pg_attribute.attrelid)
|
JOIN pg_attribute ON (pc.oid = pg_attribute.attrelid)
|
||||||
ORDER BY attnum;
|
ORDER BY attnum;
|
||||||
attname | atttypid
|
attname | atttypid
|
||||||
-------------------------------+-------------------
|
---------------------------------------------------------------------
|
||||||
tableoid | oid
|
tableoid | oid
|
||||||
cmax | cid
|
cmax | cid
|
||||||
xmax | xid
|
xmax | xid
|
||||||
|
@ -791,7 +787,7 @@ ORDER BY attnum;
|
||||||
-- verify that we can rename distributed tables
|
-- verify that we can rename distributed tables
|
||||||
SHOW citus.enable_ddl_propagation;
|
SHOW citus.enable_ddl_propagation;
|
||||||
citus.enable_ddl_propagation
|
citus.enable_ddl_propagation
|
||||||
------------------------------
|
---------------------------------------------------------------------
|
||||||
on
|
on
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -799,7 +795,7 @@ ALTER TABLE lineitem_alter RENAME TO lineitem_renamed;
|
||||||
-- verify rename is performed
|
-- verify rename is performed
|
||||||
SELECT relname FROM pg_class WHERE relname = 'lineitem_renamed';
|
SELECT relname FROM pg_class WHERE relname = 'lineitem_renamed';
|
||||||
relname
|
relname
|
||||||
------------------
|
---------------------------------------------------------------------
|
||||||
lineitem_renamed
|
lineitem_renamed
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -807,7 +803,7 @@ SELECT relname FROM pg_class WHERE relname = 'lineitem_renamed';
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname FROM pg_class WHERE relname LIKE 'lineitem_renamed%' ORDER BY relname;
|
SELECT relname FROM pg_class WHERE relname LIKE 'lineitem_renamed%' ORDER BY relname;
|
||||||
relname
|
relname
|
||||||
-------------------------
|
---------------------------------------------------------------------
|
||||||
lineitem_renamed_220000
|
lineitem_renamed_220000
|
||||||
lineitem_renamed_220001
|
lineitem_renamed_220001
|
||||||
lineitem_renamed_220003
|
lineitem_renamed_220003
|
||||||
|
@ -820,7 +816,7 @@ ALTER TABLE lineitem_renamed RENAME TO lineitem_alter;
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname FROM pg_class WHERE relname LIKE 'lineitem_alter%' AND relname <> 'lineitem_alter_220002' /* failed copy trails */ ORDER BY relname;
|
SELECT relname FROM pg_class WHERE relname LIKE 'lineitem_alter%' AND relname <> 'lineitem_alter_220002' /* failed copy trails */ ORDER BY relname;
|
||||||
relname
|
relname
|
||||||
-----------------------
|
---------------------------------------------------------------------
|
||||||
lineitem_alter_220000
|
lineitem_alter_220000
|
||||||
lineitem_alter_220001
|
lineitem_alter_220001
|
||||||
lineitem_alter_220003
|
lineitem_alter_220003
|
||||||
|
@ -831,14 +827,14 @@ SELECT relname FROM pg_class WHERE relname LIKE 'lineitem_alter%' AND relname <>
|
||||||
ALTER TABLE lineitem_alter SET(fillfactor=40);
|
ALTER TABLE lineitem_alter SET(fillfactor=40);
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname = 'lineitem_alter';
|
SELECT relname, reloptions FROM pg_class WHERE relname = 'lineitem_alter';
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
----------------+-----------------
|
---------------------------------------------------------------------
|
||||||
lineitem_alter | {fillfactor=40}
|
lineitem_alter | {fillfactor=40}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'lineitem_alter%' AND relname <> 'lineitem_alter_220002' /* failed copy trails */ ORDER BY relname;
|
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'lineitem_alter%' AND relname <> 'lineitem_alter_220002' /* failed copy trails */ ORDER BY relname;
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
-----------------------+-----------------
|
---------------------------------------------------------------------
|
||||||
lineitem_alter_220000 | {fillfactor=40}
|
lineitem_alter_220000 | {fillfactor=40}
|
||||||
lineitem_alter_220001 | {fillfactor=40}
|
lineitem_alter_220001 | {fillfactor=40}
|
||||||
lineitem_alter_220003 | {fillfactor=40}
|
lineitem_alter_220003 | {fillfactor=40}
|
||||||
|
@ -848,14 +844,14 @@ SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'lineitem_alter%' AN
|
||||||
ALTER TABLE lineitem_alter RESET(fillfactor);
|
ALTER TABLE lineitem_alter RESET(fillfactor);
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname = 'lineitem_alter';
|
SELECT relname, reloptions FROM pg_class WHERE relname = 'lineitem_alter';
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
----------------+------------
|
---------------------------------------------------------------------
|
||||||
lineitem_alter |
|
lineitem_alter |
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'lineitem_alter%' AND relname <> 'lineitem_alter_220002' /* failed copy trails */ ORDER BY relname;
|
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'lineitem_alter%' AND relname <> 'lineitem_alter_220002' /* failed copy trails */ ORDER BY relname;
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
-----------------------+------------
|
---------------------------------------------------------------------
|
||||||
lineitem_alter_220000 |
|
lineitem_alter_220000 |
|
||||||
lineitem_alter_220001 |
|
lineitem_alter_220001 |
|
||||||
lineitem_alter_220003 |
|
lineitem_alter_220003 |
|
||||||
|
@ -868,7 +864,7 @@ ALTER INDEX temp_index_1 RENAME TO idx_lineitem_linenumber;
|
||||||
-- verify rename is performed
|
-- verify rename is performed
|
||||||
SELECT relname FROM pg_class WHERE relname = 'idx_lineitem_linenumber';
|
SELECT relname FROM pg_class WHERE relname = 'idx_lineitem_linenumber';
|
||||||
relname
|
relname
|
||||||
-------------------------
|
---------------------------------------------------------------------
|
||||||
idx_lineitem_linenumber
|
idx_lineitem_linenumber
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -876,7 +872,7 @@ SELECT relname FROM pg_class WHERE relname = 'idx_lineitem_linenumber';
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname FROM pg_class WHERE relname LIKE 'idx_lineitem_linenumber%' ORDER BY relname;
|
SELECT relname FROM pg_class WHERE relname LIKE 'idx_lineitem_linenumber%' ORDER BY relname;
|
||||||
relname
|
relname
|
||||||
--------------------------------
|
---------------------------------------------------------------------
|
||||||
idx_lineitem_linenumber_220000
|
idx_lineitem_linenumber_220000
|
||||||
idx_lineitem_linenumber_220001
|
idx_lineitem_linenumber_220001
|
||||||
idx_lineitem_linenumber_220003
|
idx_lineitem_linenumber_220003
|
||||||
|
@ -892,7 +888,7 @@ ALTER TABLE lineitem_alter RENAME TO lineitem_renamed;
|
||||||
-- verify rename is performed
|
-- verify rename is performed
|
||||||
SELECT relname FROM pg_class WHERE relname = 'lineitem_alter' or relname = 'lineitem_renamed';
|
SELECT relname FROM pg_class WHERE relname = 'lineitem_alter' or relname = 'lineitem_renamed';
|
||||||
relname
|
relname
|
||||||
------------------
|
---------------------------------------------------------------------
|
||||||
lineitem_renamed
|
lineitem_renamed
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -904,15 +900,13 @@ ALTER TABLE lineitem_alter ADD COLUMN column_only_added_to_master int;
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT column_only_added_to_master FROM lineitem_alter_220000 LIMIT 0;
|
SELECT column_only_added_to_master FROM lineitem_alter_220000 LIMIT 0;
|
||||||
ERROR: column "column_only_added_to_master" does not exist
|
ERROR: column "column_only_added_to_master" does not exist
|
||||||
LINE 1: SELECT column_only_added_to_master FROM lineitem_alter_22000...
|
|
||||||
^
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
-- ddl propagation flag is reset to default, disable it again
|
-- ddl propagation flag is reset to default, disable it again
|
||||||
SET citus.enable_ddl_propagation to false;
|
SET citus.enable_ddl_propagation to false;
|
||||||
-- following query succeeds since it accesses an previously existing column
|
-- following query succeeds since it accesses an previously existing column
|
||||||
SELECT l_orderkey FROM lineitem_alter LIMIT 0;
|
SELECT l_orderkey FROM lineitem_alter LIMIT 0;
|
||||||
l_orderkey
|
l_orderkey
|
||||||
------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
-- make master and workers have the same schema again
|
-- make master and workers have the same schema again
|
||||||
|
@ -920,7 +914,7 @@ ALTER TABLE lineitem_alter DROP COLUMN column_only_added_to_master;
|
||||||
-- now this should succeed
|
-- now this should succeed
|
||||||
SELECT * FROM lineitem_alter LIMIT 0;
|
SELECT * FROM lineitem_alter LIMIT 0;
|
||||||
l_orderkey | l_partkey | l_suppkey | l_linenumber | l_quantity | l_extendedprice | l_discount | l_tax | l_returnflag | l_linestatus | l_shipdate | l_commitdate | l_receiptdate | l_shipinstruct | l_shipmode | l_comment | null_column
|
l_orderkey | l_partkey | l_suppkey | l_linenumber | l_quantity | l_extendedprice | l_discount | l_tax | l_returnflag | l_linestatus | l_shipdate | l_commitdate | l_receiptdate | l_shipinstruct | l_shipmode | l_comment | null_column
|
||||||
------------+-----------+-----------+--------------+------------+-----------------+------------+-------+--------------+--------------+------------+--------------+---------------+----------------+------------+-----------+-------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
-- previously unsupported statements are accepted by postgresql now
|
-- previously unsupported statements are accepted by postgresql now
|
||||||
|
@ -936,7 +930,7 @@ ERROR: cannot execute ALTER TABLE command dropping partition column
|
||||||
CREATE UNIQUE INDEX unique_lineitem_partkey on lineitem_alter(l_partkey);
|
CREATE UNIQUE INDEX unique_lineitem_partkey on lineitem_alter(l_partkey);
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
-------------------------+----------------
|
---------------------------------------------------------------------
|
||||||
unique_lineitem_partkey | lineitem_alter
|
unique_lineitem_partkey | lineitem_alter
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -944,7 +938,7 @@ SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT indexname, tablename FROM pg_indexes WHERE tablename like 'lineitem_alter_%';
|
SELECT indexname, tablename FROM pg_indexes WHERE tablename like 'lineitem_alter_%';
|
||||||
indexname | tablename
|
indexname | tablename
|
||||||
-----------+-----------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
@ -954,7 +948,7 @@ SET citus.shard_replication_factor TO 2;
|
||||||
CREATE TABLE sequence_deadlock_test (a serial, b serial);
|
CREATE TABLE sequence_deadlock_test (a serial, b serial);
|
||||||
SELECT create_distributed_table('sequence_deadlock_test', 'a');
|
SELECT create_distributed_table('sequence_deadlock_test', 'a');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -975,7 +969,7 @@ CREATE TABLE trigger_table (
|
||||||
);
|
);
|
||||||
SELECT create_distributed_table('trigger_table', 'id');
|
SELECT create_distributed_table('trigger_table', 'id');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -994,7 +988,7 @@ FOR EACH ROW EXECUTE PROCEDURE update_value();
|
||||||
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
|
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
|
||||||
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
|
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
|
||||||
value | count
|
value | count
|
||||||
-----------------+-------
|
---------------------------------------------------------------------
|
||||||
trigger enabled | 1
|
trigger enabled | 1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -1002,18 +996,18 @@ ALTER TABLE trigger_table DISABLE TRIGGER ALL;
|
||||||
ERROR: triggers are only supported for local tables added to metadata
|
ERROR: triggers are only supported for local tables added to metadata
|
||||||
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
|
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
|
||||||
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
|
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
|
||||||
value | count
|
value | count
|
||||||
------------------+-------
|
---------------------------------------------------------------------
|
||||||
trigger enabled | 2
|
trigger enabled | 2
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
ALTER TABLE trigger_table ENABLE TRIGGER ALL;
|
ALTER TABLE trigger_table ENABLE TRIGGER ALL;
|
||||||
ERROR: triggers are only supported for local tables added to metadata
|
ERROR: triggers are only supported for local tables added to metadata
|
||||||
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
|
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
|
||||||
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
|
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
|
||||||
value | count
|
value | count
|
||||||
------------------+-------
|
---------------------------------------------------------------------
|
||||||
trigger enabled | 3
|
trigger enabled | 3
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DROP TABLE trigger_table;
|
DROP TABLE trigger_table;
|
||||||
|
@ -1035,7 +1029,7 @@ DROP TABLE lineitem_alter;
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname FROM pg_class WHERE relname LIKE 'lineitem_alter%';
|
SELECT relname FROM pg_class WHERE relname LIKE 'lineitem_alter%';
|
||||||
relname
|
relname
|
||||||
-----------------------
|
---------------------------------------------------------------------
|
||||||
lineitem_alter_220002
|
lineitem_alter_220002
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -1045,7 +1039,7 @@ BEGIN;
|
||||||
CREATE TABLE test_table_1(id int);
|
CREATE TABLE test_table_1(id int);
|
||||||
SELECT create_distributed_table('test_table_1','id');
|
SELECT create_distributed_table('test_table_1','id');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -1056,7 +1050,7 @@ END;
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname FROM pg_class WHERE relname LIKE 'test_table_1%';
|
SELECT relname FROM pg_class WHERE relname LIKE 'test_table_1%';
|
||||||
relname
|
relname
|
||||||
---------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
@ -1065,14 +1059,14 @@ CREATE TABLE logged_test(id int);
|
||||||
ALTER TABLE logged_test SET UNLOGGED;
|
ALTER TABLE logged_test SET UNLOGGED;
|
||||||
SELECT create_distributed_table('logged_test', 'id');
|
SELECT create_distributed_table('logged_test', 'id');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
||||||
relname | logged_info
|
relname | logged_info
|
||||||
--------------------+-------------
|
---------------------------------------------------------------------
|
||||||
logged_test_220022 | unlogged
|
logged_test_220022 | unlogged
|
||||||
logged_test_220023 | unlogged
|
logged_test_220023 | unlogged
|
||||||
logged_test_220024 | unlogged
|
logged_test_220024 | unlogged
|
||||||
|
@ -1084,14 +1078,14 @@ SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logg
|
||||||
ALTER TABLE logged_test SET LOGGED;
|
ALTER TABLE logged_test SET LOGGED;
|
||||||
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
||||||
relname | logged_info
|
relname | logged_info
|
||||||
-------------+-------------
|
---------------------------------------------------------------------
|
||||||
logged_test | logged
|
logged_test | logged
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
||||||
relname | logged_info
|
relname | logged_info
|
||||||
--------------------+-------------
|
---------------------------------------------------------------------
|
||||||
logged_test_220022 | logged
|
logged_test_220022 | logged
|
||||||
logged_test_220023 | logged
|
logged_test_220023 | logged
|
||||||
logged_test_220024 | logged
|
logged_test_220024 | logged
|
||||||
|
@ -1102,14 +1096,14 @@ SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logg
|
||||||
ALTER TABLE logged_test SET UNLOGGED;
|
ALTER TABLE logged_test SET UNLOGGED;
|
||||||
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
||||||
relname | logged_info
|
relname | logged_info
|
||||||
-------------+-------------
|
---------------------------------------------------------------------
|
||||||
logged_test | unlogged
|
logged_test | unlogged
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
SELECT relname, CASE relpersistence WHEN 'u' THEN 'unlogged' WHEN 'p' then 'logged' ELSE 'unknown' END AS logged_info FROM pg_class WHERE relname ~ 'logged_test*' ORDER BY relname;
|
||||||
relname | logged_info
|
relname | logged_info
|
||||||
--------------------+-------------
|
---------------------------------------------------------------------
|
||||||
logged_test_220022 | unlogged
|
logged_test_220022 | unlogged
|
||||||
logged_test_220023 | unlogged
|
logged_test_220023 | unlogged
|
||||||
logged_test_220024 | unlogged
|
logged_test_220024 | unlogged
|
||||||
|
@ -1122,21 +1116,21 @@ DROP TABLE logged_test;
|
||||||
CREATE TABLE hash_dist(id bigint primary key, f1 text) WITH (fillfactor=40);
|
CREATE TABLE hash_dist(id bigint primary key, f1 text) WITH (fillfactor=40);
|
||||||
SELECT create_distributed_table('hash_dist','id');
|
SELECT create_distributed_table('hash_dist','id');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- verify that the storage options made it to the table definitions
|
-- verify that the storage options made it to the table definitions
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname = 'hash_dist';
|
SELECT relname, reloptions FROM pg_class WHERE relname = 'hash_dist';
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
-----------+-----------------
|
---------------------------------------------------------------------
|
||||||
hash_dist | {fillfactor=40}
|
hash_dist | {fillfactor=40}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relkind = 'r' AND relname LIKE 'hash_dist%' ORDER BY relname;
|
SELECT relname, reloptions FROM pg_class WHERE relkind = 'r' AND relname LIKE 'hash_dist%' ORDER BY relname;
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
------------------+-----------------
|
---------------------------------------------------------------------
|
||||||
hash_dist_220026 | {fillfactor=40}
|
hash_dist_220026 | {fillfactor=40}
|
||||||
hash_dist_220027 | {fillfactor=40}
|
hash_dist_220027 | {fillfactor=40}
|
||||||
hash_dist_220028 | {fillfactor=40}
|
hash_dist_220028 | {fillfactor=40}
|
||||||
|
@ -1148,14 +1142,14 @@ SELECT relname, reloptions FROM pg_class WHERE relkind = 'r' AND relname LIKE 'h
|
||||||
ALTER INDEX hash_dist_pkey SET(fillfactor=40);
|
ALTER INDEX hash_dist_pkey SET(fillfactor=40);
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname = 'hash_dist_pkey';
|
SELECT relname, reloptions FROM pg_class WHERE relname = 'hash_dist_pkey';
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
----------------+-----------------
|
---------------------------------------------------------------------
|
||||||
hash_dist_pkey | {fillfactor=40}
|
hash_dist_pkey | {fillfactor=40}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'hash_dist_pkey%' ORDER BY relname;
|
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'hash_dist_pkey%' ORDER BY relname;
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
-----------------------+-----------------
|
---------------------------------------------------------------------
|
||||||
hash_dist_pkey_220026 | {fillfactor=40}
|
hash_dist_pkey_220026 | {fillfactor=40}
|
||||||
hash_dist_pkey_220027 | {fillfactor=40}
|
hash_dist_pkey_220027 | {fillfactor=40}
|
||||||
hash_dist_pkey_220028 | {fillfactor=40}
|
hash_dist_pkey_220028 | {fillfactor=40}
|
||||||
|
@ -1166,14 +1160,14 @@ SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'hash_dist_pkey%' OR
|
||||||
ALTER INDEX hash_dist_pkey RESET(fillfactor);
|
ALTER INDEX hash_dist_pkey RESET(fillfactor);
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname = 'hash_dist_pkey';
|
SELECT relname, reloptions FROM pg_class WHERE relname = 'hash_dist_pkey';
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
----------------+------------
|
---------------------------------------------------------------------
|
||||||
hash_dist_pkey |
|
hash_dist_pkey |
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'hash_dist_pkey%' ORDER BY relname;
|
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'hash_dist_pkey%' ORDER BY relname;
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
-----------------------+------------
|
---------------------------------------------------------------------
|
||||||
hash_dist_pkey_220026 |
|
hash_dist_pkey_220026 |
|
||||||
hash_dist_pkey_220027 |
|
hash_dist_pkey_220027 |
|
||||||
hash_dist_pkey_220028 |
|
hash_dist_pkey_220028 |
|
||||||
|
@ -1190,14 +1184,14 @@ CREATE UNIQUE INDEX another_index ON hash_dist(id) WITH (fillfactor=50);
|
||||||
-- show the index and its storage options on coordinator, then workers
|
-- show the index and its storage options on coordinator, then workers
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname = 'another_index';
|
SELECT relname, reloptions FROM pg_class WHERE relname = 'another_index';
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
---------------+-----------------
|
---------------------------------------------------------------------
|
||||||
another_index | {fillfactor=50}
|
another_index | {fillfactor=50}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'another_index%' ORDER BY relname;
|
SELECT relname, reloptions FROM pg_class WHERE relname LIKE 'another_index%' ORDER BY relname;
|
||||||
relname | reloptions
|
relname | reloptions
|
||||||
----------------------+-----------------
|
---------------------------------------------------------------------
|
||||||
another_index_220026 | {fillfactor=50}
|
another_index_220026 | {fillfactor=50}
|
||||||
another_index_220027 | {fillfactor=50}
|
another_index_220027 | {fillfactor=50}
|
||||||
another_index_220028 | {fillfactor=50}
|
another_index_220028 | {fillfactor=50}
|
||||||
|
@ -1214,7 +1208,7 @@ SET citus.shard_replication_factor TO 1;
|
||||||
CREATE TABLE test_table_1(id int);
|
CREATE TABLE test_table_1(id int);
|
||||||
SELECT create_distributed_table('test_table_1', 'id');
|
SELECT create_distributed_table('test_table_1', 'id');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -1231,7 +1225,7 @@ HINT: You can issue each command separately such as ALTER TABLE test_table_1 AD
|
||||||
CREATE TABLE reference_table(i int UNIQUE);
|
CREATE TABLE reference_table(i int UNIQUE);
|
||||||
SELECT create_reference_table('reference_table');
|
SELECT create_reference_table('reference_table');
|
||||||
create_reference_table
|
create_reference_table
|
||||||
------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
@ -1247,7 +1241,7 @@ DROP TABLE reference_table;
|
||||||
CREATE TABLE referenced_table(i int UNIQUE);
|
CREATE TABLE referenced_table(i int UNIQUE);
|
||||||
SELECT create_distributed_table('referenced_table', 'i');
|
SELECT create_distributed_table('referenced_table', 'i');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
--------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
|
|
@ -97,9 +97,9 @@ SELECT count(*) FROM single_replicatated;
|
||||||
|
|
||||||
SET citus.force_max_query_parallelization TO OFF;
|
SET citus.force_max_query_parallelization TO OFF;
|
||||||
|
|
||||||
-- one similar test, but this time on modification queries
|
-- one similar test, and this time on modification queries
|
||||||
-- to see that connection establishement failures could
|
-- to see that connection establishement failures could
|
||||||
-- mark placement INVALID
|
-- fail the transaction (but not mark any placements as INVALID)
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SELECT
|
SELECT
|
||||||
|
@ -120,7 +120,7 @@ WHERE
|
||||||
shardstate = 3 AND
|
shardstate = 3 AND
|
||||||
shardid IN (SELECT shardid from pg_dist_shard where logicalrelid = 'products'::regclass);
|
shardid IN (SELECT shardid from pg_dist_shard where logicalrelid = 'products'::regclass);
|
||||||
|
|
||||||
-- show that INSERT went through
|
-- show that INSERT failed
|
||||||
SELECT count(*) FROM products WHERE product_no = 100;
|
SELECT count(*) FROM products WHERE product_no = 100;
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -210,7 +210,7 @@ COPY dml_test FROM STDIN WITH CSV;
|
||||||
|
|
||||||
---- test multiple statements against a single shard, but with two placements
|
---- test multiple statements against a single shard, but with two placements
|
||||||
|
|
||||||
-- fail at COMMIT (actually COMMIT this time, as no 2pc in use)
|
-- fail at PREPARED COMMIT as we use 2PC
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
@ -221,14 +221,19 @@ UPDATE dml_test SET name = 'alpha' WHERE id = 1;
|
||||||
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
|
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
--- should see all changes, but they only went to one placement (other is unhealthy)
|
-- all changes should be committed because we injected
|
||||||
SELECT * FROM dml_test ORDER BY id ASC;
|
-- the failure on the COMMIT time. And, we should not
|
||||||
|
-- mark any placements as INVALID
|
||||||
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
|
SELECT recover_prepared_transactions();
|
||||||
SELECT shardid FROM pg_dist_shard_placement WHERE shardstate = 3;
|
SELECT shardid FROM pg_dist_shard_placement WHERE shardstate = 3;
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SET citus.task_assignment_policy TO "round-robin";
|
||||||
|
SELECT * FROM dml_test ORDER BY id ASC;
|
||||||
|
SELECT * FROM dml_test ORDER BY id ASC;
|
||||||
|
RESET citus.task_assignment_policy;
|
||||||
|
|
||||||
-- drop table and recreate as reference table
|
-- drop table and recreate as reference table
|
||||||
|
|
||||||
DROP TABLE dml_test;
|
DROP TABLE dml_test;
|
||||||
SET citus.shard_count = 2;
|
SET citus.shard_count = 2;
|
||||||
SET citus.shard_replication_factor = 1;
|
SET citus.shard_replication_factor = 1;
|
||||||
|
|
|
@ -13,14 +13,14 @@ INSERT INTO mod_test VALUES (2, 6);
|
||||||
|
|
||||||
SELECT COUNT(*) FROM mod_test WHERE key=2;
|
SELECT COUNT(*) FROM mod_test WHERE key=2;
|
||||||
|
|
||||||
-- some clean up
|
-- none of the placements are marked as INACTIVE
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
UPDATE pg_dist_shard_placement SET shardstate = 1
|
||||||
WHERE shardid IN (
|
WHERE shardid IN (
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
||||||
) AND shardstate = 3 RETURNING placementid;
|
) AND shardstate = 3 RETURNING placementid;
|
||||||
TRUNCATE mod_test;
|
TRUNCATE mod_test;
|
||||||
|
|
||||||
-- verify behavior of UPDATE ... RETURNING; should mark as failed
|
-- verify behavior of UPDATE ... RETURNING; should fail the transaction
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
INSERT INTO mod_test VALUES (2, 6);
|
INSERT INTO mod_test VALUES (2, 6);
|
||||||
|
|
||||||
|
@ -29,7 +29,7 @@ UPDATE mod_test SET value='ok' WHERE key=2 RETURNING key;
|
||||||
|
|
||||||
SELECT COUNT(*) FROM mod_test WHERE value='ok';
|
SELECT COUNT(*) FROM mod_test WHERE value='ok';
|
||||||
|
|
||||||
-- some clean up
|
-- none of the placements are marked as INACTIVE
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
UPDATE pg_dist_shard_placement SET shardstate = 1
|
||||||
WHERE shardid IN (
|
WHERE shardid IN (
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
||||||
|
@ -37,7 +37,7 @@ WHERE shardid IN (
|
||||||
TRUNCATE mod_test;
|
TRUNCATE mod_test;
|
||||||
|
|
||||||
-- verify behavior of multi-statement modifications to a single shard
|
-- verify behavior of multi-statement modifications to a single shard
|
||||||
-- should succeed but mark a placement as failed
|
-- should fail the transaction and never mark placements inactive
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
@ -49,7 +49,7 @@ COMMIT;
|
||||||
|
|
||||||
SELECT COUNT(*) FROM mod_test WHERE key=2;
|
SELECT COUNT(*) FROM mod_test WHERE key=2;
|
||||||
|
|
||||||
-- some clean up
|
-- none of the placements are marked as INACTIVE
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
UPDATE pg_dist_shard_placement SET shardstate = 1
|
||||||
WHERE shardid IN (
|
WHERE shardid IN (
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
|
||||||
|
|
|
@ -14,21 +14,15 @@ SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").kill()');
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
SELECT * FROM select_test WHERE key = 3;
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
SELECT * FROM select_test WHERE key = 3;
|
||||||
|
|
||||||
-- kill after first SELECT; txn should work (though placement marked bad)
|
-- kill after first SELECT; txn should fail as INSERT triggers
|
||||||
|
-- 2PC (and placementis not marked bad)
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").kill()');
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO select_test VALUES (3, 'more data');
|
INSERT INTO select_test VALUES (3, 'more data');
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
SELECT * FROM select_test WHERE key = 3;
|
||||||
INSERT INTO select_test VALUES (3, 'even more data');
|
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
-- some clean up
|
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
|
||||||
WHERE shardid IN (
|
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'select_test'::regclass
|
|
||||||
);
|
|
||||||
TRUNCATE select_test;
|
TRUNCATE select_test;
|
||||||
|
|
||||||
-- now the same tests with query cancellation
|
-- now the same tests with query cancellation
|
||||||
|
@ -66,7 +60,7 @@ INSERT INTO select_test VALUES (3, 'even more data');
|
||||||
SELECT * FROM select_test WHERE key = 3;
|
SELECT * FROM select_test WHERE key = 3;
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
-- error after second SELECT; txn should work (though placement marked bad)
|
-- error after second SELECT; txn should fails the transaction
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(1).reset()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(1).reset()');
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
|
@ -24,16 +24,14 @@ ANALYZE vacuum_test;
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
|
||||||
ANALYZE vacuum_test;
|
ANALYZE vacuum_test;
|
||||||
|
|
||||||
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
|
SELECT recover_prepared_transactions();
|
||||||
|
|
||||||
-- ANALYZE transactions being critical is an open question, see #2430
|
-- ANALYZE transactions being critical is an open question, see #2430
|
||||||
-- show that we marked as INVALID on COMMIT FAILURE
|
-- show that we never mark as INVALID on COMMIT FAILURE
|
||||||
SELECT shardid, shardstate FROM pg_dist_shard_placement where shardstate != 1 AND
|
SELECT shardid, shardstate FROM pg_dist_shard_placement where shardstate != 1 AND
|
||||||
shardid in ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass);
|
shardid in ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass);
|
||||||
|
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1
|
|
||||||
WHERE shardid IN (
|
|
||||||
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass
|
|
||||||
);
|
|
||||||
|
|
||||||
-- the same tests with cancel
|
-- the same tests with cancel
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM").cancel(' || pg_backend_pid() || ')');
|
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM").cancel(' || pg_backend_pid() || ')');
|
||||||
VACUUM vacuum_test;
|
VACUUM vacuum_test;
|
||||||
|
|
|
@ -1,5 +1,10 @@
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
|
||||||
|
SET citus.shard_replication_factor TO 2;
|
||||||
|
CREATE TABLE the_replicated_table (a int, b int, z bigserial);
|
||||||
|
SELECT create_distributed_table('the_replicated_table', 'a');
|
||||||
|
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
CREATE TABLE the_table (a int, b int, z bigserial);
|
CREATE TABLE the_table (a int, b int, z bigserial);
|
||||||
SELECT create_distributed_table('the_table', 'a');
|
SELECT create_distributed_table('the_table', 'a');
|
||||||
|
|
||||||
|
@ -21,20 +26,27 @@ INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
|
||||||
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
|
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
|
||||||
|
|
||||||
-- We can allow DML on a writable standby coordinator.
|
-- We can allow DML on a writable standby coordinator.
|
||||||
-- Note that it doesn't help to enable writes for citus local tables
|
-- Note that it doesn't help to enable writes for
|
||||||
-- and coordinator replicated reference tables. This is because, the
|
-- (a) citus local tables
|
||||||
-- data is in the coordinator and will hit read-only tranaction checks
|
-- (b) coordinator replicated reference tables.
|
||||||
-- on Postgres
|
-- (c) reference tables or replication > 1 distributed tables
|
||||||
|
-- (a) and (b) is because the data is in the coordinator and will hit
|
||||||
|
-- read-only tranaction checks on Postgres
|
||||||
|
-- (c) is because citus uses 2PC, where a transaction record should
|
||||||
|
-- be inserted to pg_dist_node, which is not allowed
|
||||||
SET citus.writable_standby_coordinator TO on;
|
SET citus.writable_standby_coordinator TO on;
|
||||||
|
|
||||||
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
|
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
|
||||||
SELECT * FROM the_table;
|
SELECT * FROM the_table;
|
||||||
|
INSERT INTO the_replicated_table (a, b, z) VALUES (1, 2, 2);
|
||||||
|
SELECT * FROM the_replicated_table;
|
||||||
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
|
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
|
||||||
SELECT * FROM reference_table;
|
SELECT * FROM reference_table;
|
||||||
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
|
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
|
||||||
SELECT * FROM citus_local_table;
|
SELECT * FROM citus_local_table;
|
||||||
|
|
||||||
UPDATE the_table SET z = 3 WHERE a = 1;
|
UPDATE the_table SET z = 3 WHERE a = 1;
|
||||||
|
UPDATE the_replicated_table SET z = 3 WHERE a = 1;
|
||||||
UPDATE reference_table SET z = 3 WHERE a = 1;
|
UPDATE reference_table SET z = 3 WHERE a = 1;
|
||||||
UPDATE citus_local_table SET z = 3 WHERE a = 1;
|
UPDATE citus_local_table SET z = 3 WHERE a = 1;
|
||||||
SELECT * FROM the_table;
|
SELECT * FROM the_table;
|
||||||
|
@ -42,6 +54,7 @@ SELECT * FROM reference_table;
|
||||||
SELECT * FROM citus_local_table;
|
SELECT * FROM citus_local_table;
|
||||||
|
|
||||||
DELETE FROM the_table WHERE a = 1;
|
DELETE FROM the_table WHERE a = 1;
|
||||||
|
DELETE FROM the_replicated_table WHERE a = 1;
|
||||||
DELETE FROM reference_table WHERE a = 1;
|
DELETE FROM reference_table WHERE a = 1;
|
||||||
DELETE FROM citus_local_table WHERE a = 1;
|
DELETE FROM citus_local_table WHERE a = 1;
|
||||||
|
|
||||||
|
@ -51,11 +64,13 @@ SELECT * FROM citus_local_table;
|
||||||
|
|
||||||
-- drawing from a sequence is not possible
|
-- drawing from a sequence is not possible
|
||||||
INSERT INTO the_table (a, b) VALUES (1, 2);
|
INSERT INTO the_table (a, b) VALUES (1, 2);
|
||||||
|
INSERT INTO the_replicated_table (a, b) VALUES (1, 2);
|
||||||
INSERT INTO reference_table (a, b) VALUES (1, 2);
|
INSERT INTO reference_table (a, b) VALUES (1, 2);
|
||||||
INSERT INTO citus_local_table (a, b) VALUES (1, 2);
|
INSERT INTO citus_local_table (a, b) VALUES (1, 2);
|
||||||
|
|
||||||
-- 2PC is not possible
|
-- 2PC is not possible
|
||||||
INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
|
INSERT INTO the_replicated_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
INSERT INTO citus_local_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
INSERT INTO citus_local_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
|
||||||
|
|
||||||
|
@ -64,6 +79,11 @@ COPY the_table (a, b, z) FROM STDIN WITH CSV;
|
||||||
10,10,10
|
10,10,10
|
||||||
11,11,11
|
11,11,11
|
||||||
\.
|
\.
|
||||||
|
-- COPY is not possible in 2PC mode
|
||||||
|
COPY the_replicated_table (a, b, z) FROM STDIN WITH CSV;
|
||||||
|
10,10,10
|
||||||
|
11,11,11
|
||||||
|
\.
|
||||||
COPY reference_table (a, b, z) FROM STDIN WITH CSV;
|
COPY reference_table (a, b, z) FROM STDIN WITH CSV;
|
||||||
10,10,10
|
10,10,10
|
||||||
11,11,11
|
11,11,11
|
||||||
|
@ -87,6 +107,8 @@ WITH del AS (DELETE FROM the_table RETURNING *)
|
||||||
SELECT * FROM del ORDER BY a;
|
SELECT * FROM del ORDER BY a;
|
||||||
WITH del AS (DELETE FROM reference_table RETURNING *)
|
WITH del AS (DELETE FROM reference_table RETURNING *)
|
||||||
SELECT * FROM del ORDER BY a;
|
SELECT * FROM del ORDER BY a;
|
||||||
|
WITH del AS (DELETE FROM the_replicated_table RETURNING *)
|
||||||
|
SELECT * FROM del ORDER BY a;
|
||||||
WITH del AS (DELETE FROM citus_local_table RETURNING *)
|
WITH del AS (DELETE FROM citus_local_table RETURNING *)
|
||||||
SELECT * FROM del ORDER BY a;
|
SELECT * FROM del ORDER BY a;
|
||||||
|
|
||||||
|
|
|
@ -301,7 +301,8 @@ ORDER BY nodeport, shardid;
|
||||||
|
|
||||||
-- hide postgresql version dependend messages for next test only
|
-- hide postgresql version dependend messages for next test only
|
||||||
\set VERBOSITY terse
|
\set VERBOSITY terse
|
||||||
-- deferred check should abort the transaction
|
-- for replicated tables use 2PC even if multi-shard commit protocol
|
||||||
|
-- is set to 2PC
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SET LOCAL citus.multi_shard_commit_protocol TO '1pc';
|
SET LOCAL citus.multi_shard_commit_protocol TO '1pc';
|
||||||
DELETE FROM researchers WHERE lab_id = 6;
|
DELETE FROM researchers WHERE lab_id = 6;
|
||||||
|
@ -487,16 +488,17 @@ FOR EACH ROW EXECUTE PROCEDURE reject_bad();
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
|
||||||
-- should be the same story as before, just at COMMIT time
|
-- should be the same story as before, just at COMMIT time
|
||||||
|
-- as we use 2PC, the transaction is rollbacked
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO objects VALUES (1, 'apple');
|
INSERT INTO objects VALUES (1, 'apple');
|
||||||
INSERT INTO objects VALUES (2, 'BAD');
|
INSERT INTO objects VALUES (2, 'BAD');
|
||||||
INSERT INTO labs VALUES (9, 'Umbrella Corporation');
|
INSERT INTO labs VALUES (9, 'Umbrella Corporation');
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
-- data should be persisted
|
-- data should not persisted
|
||||||
SELECT * FROM objects WHERE id = 2;
|
SELECT * FROM objects WHERE id = 2;
|
||||||
|
|
||||||
-- but one placement should be bad
|
-- and nonne of the placements should be bad
|
||||||
SELECT count(*)
|
SELECT count(*)
|
||||||
FROM pg_dist_shard_placement AS sp,
|
FROM pg_dist_shard_placement AS sp,
|
||||||
pg_dist_shard AS s
|
pg_dist_shard AS s
|
||||||
|
@ -560,11 +562,11 @@ INSERT INTO labs VALUES (9, 'BAD');
|
||||||
COMMIT;
|
COMMIT;
|
||||||
\set VERBOSITY default
|
\set VERBOSITY default
|
||||||
|
|
||||||
-- data to objects should be persisted, but labs should not...
|
-- none of the changes should be persisted
|
||||||
SELECT * FROM objects WHERE id = 1;
|
SELECT * FROM objects WHERE id = 1;
|
||||||
SELECT * FROM labs WHERE id = 8;
|
SELECT * FROM labs WHERE id = 8;
|
||||||
|
|
||||||
-- labs should be healthy, but one object placement shouldn't be
|
-- all placements should be healthy
|
||||||
SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*)
|
SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*)
|
||||||
FROM pg_dist_shard_placement AS sp,
|
FROM pg_dist_shard_placement AS sp,
|
||||||
pg_dist_shard AS s
|
pg_dist_shard AS s
|
||||||
|
|
|
@ -1193,16 +1193,10 @@ GRANT INSERT ON ALL TABLES IN SCHEMA public TO router_user;
|
||||||
CREATE USER router_user;
|
CREATE USER router_user;
|
||||||
GRANT INSERT ON ALL TABLES IN SCHEMA public TO router_user;
|
GRANT INSERT ON ALL TABLES IN SCHEMA public TO router_user;
|
||||||
\c - router_user - :master_port
|
\c - router_user - :master_port
|
||||||
-- first test that it is marked invalid inside a transaction block
|
|
||||||
-- we will fail to connect to worker 2, since the user does not exist
|
-- we will fail to connect to worker 2, since the user does not exist
|
||||||
|
-- still, we never mark placements inactive. Instead, fail the transaction
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO failure_test VALUES (1, 1);
|
INSERT INTO failure_test VALUES (1, 1);
|
||||||
SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement
|
|
||||||
WHERE shardid IN (
|
|
||||||
SELECT shardid FROM pg_dist_shard
|
|
||||||
WHERE logicalrelid = 'failure_test'::regclass
|
|
||||||
)
|
|
||||||
ORDER BY placementid;
|
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
INSERT INTO failure_test VALUES (2, 1);
|
INSERT INTO failure_test VALUES (2, 1);
|
||||||
SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement
|
SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement
|
||||||
|
|
|
@ -87,9 +87,10 @@ SELECT count(*) >= 4 FROM pg_dist_transaction;
|
||||||
|
|
||||||
SELECT recover_prepared_transactions();
|
SELECT recover_prepared_transactions();
|
||||||
|
|
||||||
-- plain INSERT does not use 2PC
|
-- plain INSERT uses 2PC
|
||||||
INSERT INTO test_recovery VALUES ('hello');
|
INSERT INTO test_recovery VALUES ('hello');
|
||||||
SELECT count(*) FROM pg_dist_transaction;
|
SELECT count(*) FROM pg_dist_transaction;
|
||||||
|
SELECT recover_prepared_transactions();
|
||||||
|
|
||||||
-- Aborted DDL commands should not write transaction recovery records
|
-- Aborted DDL commands should not write transaction recovery records
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
|
@ -129,12 +129,9 @@ SELECT master_create_worker_shards('second_dustbunnies', 1, 2);
|
||||||
|
|
||||||
-- run VACUUM and ANALYZE against the table on the master
|
-- run VACUUM and ANALYZE against the table on the master
|
||||||
\c - - :master_host :master_port
|
\c - - :master_host :master_port
|
||||||
SET citus.log_remote_commands TO ON;
|
|
||||||
|
|
||||||
VACUUM dustbunnies;
|
VACUUM dustbunnies;
|
||||||
ANALYZE dustbunnies;
|
ANALYZE dustbunnies;
|
||||||
|
|
||||||
|
|
||||||
-- send a VACUUM FULL and a VACUUM ANALYZE
|
-- send a VACUUM FULL and a VACUUM ANALYZE
|
||||||
|
|
||||||
VACUUM (FULL) dustbunnies;
|
VACUUM (FULL) dustbunnies;
|
||||||
|
@ -148,8 +145,6 @@ SELECT relfrozenxid AS frozenxid FROM pg_class WHERE oid='dustbunnies_990002'::r
|
||||||
|
|
||||||
-- send a VACUUM FREEZE after adding a new row
|
-- send a VACUUM FREEZE after adding a new row
|
||||||
\c - - :master_host :master_port
|
\c - - :master_host :master_port
|
||||||
SET citus.log_remote_commands TO ON;
|
|
||||||
|
|
||||||
INSERT INTO dustbunnies VALUES (5, 'peter');
|
INSERT INTO dustbunnies VALUES (5, 'peter');
|
||||||
VACUUM (FREEZE) dustbunnies;
|
VACUUM (FREEZE) dustbunnies;
|
||||||
|
|
||||||
|
@ -164,8 +159,6 @@ WHERE tablename = 'dustbunnies_990002' ORDER BY attname;
|
||||||
|
|
||||||
-- add NULL values, then perform column-specific ANALYZE
|
-- add NULL values, then perform column-specific ANALYZE
|
||||||
\c - - :master_host :master_port
|
\c - - :master_host :master_port
|
||||||
SET citus.log_remote_commands TO ON;
|
|
||||||
|
|
||||||
INSERT INTO dustbunnies VALUES (6, NULL, NULL);
|
INSERT INTO dustbunnies VALUES (6, NULL, NULL);
|
||||||
ANALYZE dustbunnies (name);
|
ANALYZE dustbunnies (name);
|
||||||
|
|
||||||
|
|
|
@ -8,7 +8,7 @@ SELECT substring(:'server_version', '\d+')::int > 13 AS server_version_above_thi
|
||||||
|
|
||||||
create schema pg14;
|
create schema pg14;
|
||||||
set search_path to pg14;
|
set search_path to pg14;
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
SET citus.next_shard_id TO 980000;
|
SET citus.next_shard_id TO 980000;
|
||||||
SET citus.shard_count TO 2;
|
SET citus.shard_count TO 2;
|
||||||
|
|
||||||
|
|
|
@ -94,7 +94,8 @@ SELECT recover_prepared_transactions();
|
||||||
ALTER TABLE test_table ADD CONSTRAINT b_check CHECK(b > 0);
|
ALTER TABLE test_table ADD CONSTRAINT b_check CHECK(b > 0);
|
||||||
SELECT distributed_2PCs_are_equal_to_placement_count();
|
SELECT distributed_2PCs_are_equal_to_placement_count();
|
||||||
|
|
||||||
-- with 1PC, we should not see and distributed TXs in the pg_dist_transaction
|
-- even if 1PC used, we use 2PC as we modify replicated tables
|
||||||
|
-- see distributed TXs in the pg_dist_transaction
|
||||||
SET citus.multi_shard_commit_protocol TO '1pc';
|
SET citus.multi_shard_commit_protocol TO '1pc';
|
||||||
SET citus.multi_shard_modify_mode TO 'sequential';
|
SET citus.multi_shard_modify_mode TO 'sequential';
|
||||||
SELECT recover_prepared_transactions();
|
SELECT recover_prepared_transactions();
|
||||||
|
@ -129,7 +130,7 @@ SET citus.shard_replication_factor TO 2;
|
||||||
CREATE TABLE test_table_rep_2 (a int);
|
CREATE TABLE test_table_rep_2 (a int);
|
||||||
SELECT create_distributed_table('test_table_rep_2', 'a');
|
SELECT create_distributed_table('test_table_rep_2', 'a');
|
||||||
|
|
||||||
-- 1PC should never use 2PC with rep > 1
|
-- even if 1PC used, we use 2PC with rep > 1
|
||||||
SET citus.multi_shard_commit_protocol TO '1pc';
|
SET citus.multi_shard_commit_protocol TO '1pc';
|
||||||
|
|
||||||
SET citus.multi_shard_modify_mode TO 'sequential';
|
SET citus.multi_shard_modify_mode TO 'sequential';
|
||||||
|
|
Loading…
Reference in New Issue