A test in background_rebalance_parallel.sql was failing intermittently
where the order of tasks in the output was not deterministic. This
commit fixes the test by removing id columns for the background tasks in
the output.
A sample failing diff before this patch is below:
```diff
SELECT D.task_id,
(SELECT T.command FROM pg_dist_background_task T WHERE T.task_id = D.task_id),
D.depends_on,
(SELECT T.command FROM pg_dist_background_task T WHERE T.task_id = D.depends_on)
FROM pg_dist_background_task_depend D WHERE job_id in (:job_id) ORDER BY D.task_id, D.depends_on ASC;
task_id | command | depends_on | command
---------+---------------------------------------------------------------------+------------+---------------------------------------------------------------------
- 1014 | SELECT pg_catalog.citus_move_shard_placement(85674026,50,57,'auto') | 1013 | SELECT pg_catalog.citus_move_shard_placement(85674025,50,56,'auto')
- 1016 | SELECT pg_catalog.citus_move_shard_placement(85674032,50,57,'auto') | 1015 | SELECT pg_catalog.citus_move_shard_placement(85674031,50,56,'auto')
- 1018 | SELECT pg_catalog.citus_move_shard_placement(85674038,50,57,'auto') | 1017 | SELECT pg_catalog.citus_move_shard_placement(85674037,50,56,'auto')
- 1020 | SELECT pg_catalog.citus_move_shard_placement(85674044,50,57,'auto') | 1019 | SELECT pg_catalog.citus_move_shard_placement(85674043,50,56,'auto')
+ 1014 | SELECT pg_catalog.citus_move_shard_placement(85674038,50,57,'auto') | 1013 | SELECT pg_catalog.citus_move_shard_placement(85674037,50,56,'auto')
+ 1016 | SELECT pg_catalog.citus_move_shard_placement(85674044,50,57,'auto') | 1015 | SELECT pg_catalog.citus_move_shard_placement(85674043,50,56,'auto')
+ 1018 | SELECT pg_catalog.citus_move_shard_placement(85674026,50,57,'auto') | 1017 | SELECT pg_catalog.citus_move_shard_placement(85674025,50,56,'auto')
+ 1020 | SELECT pg_catalog.citus_move_shard_placement(85674032,50,57,'auto') | 1019 | SELECT pg_catalog.citus_move_shard_placement(85674031,50,56,'auto')
(4 rows)
```
Notice that the dependent and dependee tasks have some commands, but
they have different task ids.
DESCRIPTION: Adds support for creating distributed tables without shard
key
Commits proposed in this PR have already been reviewed in other PRs
noted
for each commit.
With this PR, we allow creating distributed tables without
specifying a shard key via create_distributed_table(). Here are the
the important details about those tables:
* Specifying `shard_count` is not allowed because it is assumed to be 1.
* We mostly call such tables as "single-shard" distributed table in code
/ comments.
* `colocate_with` param allows colocating such single-shard tables to
each other.
* We define this table type, i.e., SINGLE_SHARD_DISTRIBUTED, as a
subclass
of DISTRIBUTED_TABLE because we mostly want to treat them as distributed
tables in terms of SQL / DDL / operation support.
* Metadata for such tables look like:
- distribution method => DISTRIBUTE_BY_NONE
- replication model => REPLICATION_MODEL_STREAMING
- colocation id => **!=** INVALID_COLOCATION_ID (distinguishes from
Citus local tables)
* We assign colocation groups for such tables to different nodes in a
round-robin fashion based on the modulo of "colocation id".
There are also still more work that needs to be done, such as improving
SQL
support, making sure that Citus operations work well such distributed
tables
and making sure that latest features merged in at 11.3 / 12.0 (such as
CDC)
works fine. We will take care of them in subsequent PRs.
In this release, we will build schema-based-sharding on top of this
infrastructure. And it's likely that we will use this infra for some
other nice features in future too.
* Add support for dist insert select by selecting from a reference
table.
This was the only pushable insert .. select case that
#6773 didn't cover.
* For the cases where we insert into a Citus table but the INSERT ..
SELECT
query cannot be pushed down, allow pull-to-coordinator when possible.
Remove the checks that we had at the very beginning of
CreateInsertSelectPlanInternal so that we can try insert .. select via
pull-to-coordinator for the cases where we cannot push-down the insert
.. select query. What we support via pull-to-coordinator is still
limited due to lacking of logical planner support for SELECT queries,
but this commit at least allows using pull-to-coordinator for the cases
where the select query can be planned via router planner, without
limiting ourselves to restrictive top-level checks.
Also introduce some additional restrictions into
CreateDistributedInsertSelectPlan for the cases it was missing to check
for null-shard-key tables. Indeed, it would make more sense to have
those checks for distributed tables in general, via separate PRs against
main branch. See https://github.com/citusdata/citus/pull/6817.
* Add support for inserting into a Postgres table.
Enable router planner and a limited version of INSERT .. SELECT planner
for the queries that reference colocated null shard key tables.
* SELECT / UPDATE / DELETE / MERGE is supported as long as it's a router
query.
* INSERT .. SELECT is supported as long as it only references colocated
null shard key tables.
Note that this is not only limited to distributed INSERT .. SELECT but
also
covers a limited set of query types that require pull-to-coordinator,
e.g.,
due to LIMIT clause, generate_series() etc. ...
(Ideally distributed INSERT .. SELECT could handle such queries too,
e.g.,
when we're only referencing tables that don't have a shard key, but
today
this is not the case. See
https://github.com/citusdata/citus/pull/6773#discussion_r1140130562.
Add tests for ddl coverage:
* indexes
* partitioned tables + indexes with long names
* triggers
* foreign keys
* statistics
* grant & revoke statements
* truncate & vacuum
* create/test/drop view that depends on a dist table with no shard key
* policy & rls test
* alter table add/drop/alter_type column (using sequences/different data
types/identity columns)
* alter table add constraint (not null, check, exclusion constraint)
* alter table add column with a default value / set default / drop
default
* alter table set option (autovacuum)
* indexes / constraints without names
* multiple subcommands
Adds support for
* Creating new partitions after distributing (with null key) the parent
table
* Attaching partitions to a distributed table with null distribution key
(and automatically distribute the new partition with null key as well)
* Detaching partitions from it
With this PR, we allow creating distributed tables with without
specifying a shard key via create_distributed_table(). Here are the
the important details about those tables:
* Specifying `shard_count` is not allowed because it is assumed to be 1.
* We mostly call such tables as "null shard-key" table in code /
comments.
* To avoid doing a breaking layout change in create_distributed_table();
instead of throwing an error, it will inform the user that
`distribution_type`
param is ignored unless it's explicitly set to NULL or 'h'.
* `colocate_with` param allows colocating such null shard-key tables to
each other.
* We define this table type, i.e., NULL_SHARD_KEY_TABLE, as a subclass
of
DISTRIBUTED_TABLE because we mostly want to treat them as distributed
tables in terms of SQL / DDL / operation support.
* Metadata for such tables look like:
- distribution method => DISTRIBUTE_BY_NONE
- replication model => REPLICATION_MODEL_STREAMING
- colocation id => **!=** INVALID_COLOCATION_ID (distinguishes from
Citus local tables)
* We assign colocation groups for such tables to different nodes in a
round-robin fashion based on the modulo of "colocation id".
Note that this PR doesn't care about DDL (except CREATE TABLE) / SQL /
operation (i.e., Citus UDFs) support for such tables but adds a
preliminary
API.
When working on changelog, Marco suggested in
https://github.com/citusdata/citus/pull/6856#pullrequestreview-1386601215
that we should bump columnar version to 11.3 as well.
This PR aims to contain all the necessary changes to allow upgrades to
and downgrades from 11.3.0 for columnar. Note that updating citus
extension version does not affect columnar as the two extension versions
are not really coupled.
The same changes will also be applied to the release branch in
https://github.com/citusdata/citus/pull/6897
In this release, I tried something different. I experimented with adding
the PR number and title to the changelog right before each changelog
entry. This way, it is easier to track where a particular changelog
entry comes from. After reviews are over, I plan to remove those lines
with PR numbers and titles.
I went through all the PRs that are merged after 11.2.0 release and came
up with a list of PRs that may need help with changelog entries. You can
see details on PRs grouped in several sections below.
## PRs with missing entries
The following PRs below do not have a changelog entry. If you think that
this is a mistake, please share it in this PR along with a suggestion on
what the changelog item should be.
PR #6846 : fix 3 flaky tests in failure schedule
PR #6844 : Add CPU usage to citus_stat_tenants
PR #6833 : Fix citus_stat_tenants period updating bug
PR #6787 : Add more tests for ddl coverage
PR #6842 : Add build-cdc-* temporary directories to .gitignore
PR #6841 : Add build-cdc-* temporary directories to .gitignore
PR #6840 : Bump Citus to 12.0devel
PR #6824 : Fixes flakiness in multi_metadata_sync test
PR #6811 : Backport identity column improvements to v11.2
PR #6830 : In run_test.py actually return worker_count
PR #6825 : Fixes flakiness in multi_cluster_management test
PR #6816 : Refactor run_test.py
PR #6817 : Explicitly disallow local rels when inserting into dist table
PR #6821 : Rename citus stats tenants
PR #6822 : Add some more tests for initial sql support
PR #6819 : Fix flakyness in
citus_split_shard_by_split_points_deferred_drop
PR #6814 : Make python-regress based tests runnable with run_test.py
PR #6813 : Fix flaky multi_mx_schema_support test
PR #6720 : Convert columnar tap tests to pytest
PR #6812 : Revoke statistics permissions from public and grant them to
pg_monitor
PR #6769 : Citus stats tenants guc
PR #6807 : Fix the incorrect (constant) value passed to pointer-to-bool
parameter, pass a NULL as the value is not used
PR #6797 : Attribute local queries and cached plans on local execution
PR #6796 : Parse the annotation string correctly
PR #6762 : Add logs to citus_stats_tenants
PR #6773 : Add initial sql support for distributed tables that don't
have a shard key
PR #6792 : Disentangle MERGE planning code from the modify-planning code
path
PR #6761 : Citus stats tenants collector view
PR #6791 : Make 8 more tests runnable multiple times via run_test.py
PR #6786 : Refactor some of the planning code to accommodate a new
planning path for MERGE SQL
PR #6789 : Rename AllRelations.. functions to AllDistributedRelations..
PR #6788 : Actually skip arbitrary_configs_router & nested_execution for
AllNullDistKeyDefaultConfig
PR #6783 : Add a config for arbitrary config tests where all the tables
are null-shard-key tables
PR #6784 : Fix attach partition: citus local to null distributed
PR #6782 : Add an arbitrary config test heavily based on
multi_router_planner_fast_path.sql
PR #6781 : Decide what to do with router planner error at one place
PR #6778 : Support partitioning for dist tables with null dist keys
PR #6766 : fix pip lock file
PR #6764 : Make workerCount configurable for regression tests
PR #6745 : Add support for creating distributed tables with a null shard
key
PR #6696 : This implements MERGE phase-III
PR #6767 : Add pytest depedencies to Pipfile
PR #6760 : Decide core distribution params in CreateCitusTable
PR #6759 : Add multi_create_fdw into minimal_schedule
PR #6743 : Replace CITUS_TABLE_WITH_NO_DIST_KEY checks with
HasDistributionKey()
PR #6751 : Stabilize single_node.sql and others that report illegal node
removal
PR #6742 : Refactor CreateDistributedTable()
PR #6747 : Remove unused lock functions
PR #6744 : Fix multiple output version arbitrary config tests
PR #6741 : Stabilize single node tests
PR #6740 : Fix string eval bug in migration files check
PR #6736 : Make run_test.py and create_test.py importable without errors
PR #6734 : Don't blanket ignore flake8 E402 error
PR #6737 : Fixes bookworm packaging pipeline problem
PR #6735 : Fix run_test.py on python 3.9
PR #6733 : MERGE: In deparser, add missing check for RETURNING clause.
PR #6714 : Remove auto_explain workaround in citus explain hook for
ALTER TABLE
PR #6719 : Fix flaky test
PR #6718 : Add more powerfull dependency tracking to run_test.py
PR #6710 : Install non-vulnerable cryptography package
PR #6711 : Support compilation and run tests on latest PG versions
PR #6700 : Add auto-formatting and linting to our python code
PR #6707 : Allow multi_insert_select to run repeatably
PR #6708 : Fix flakyness in failure_create_distributed_table_non_empty
PR #6698 : Miscellaneous cleanup
PR #6704 : Update README for 11.2
PR #6703 : Fix dubious ownership error from git
PR #6690 : Bump Citus to 11.3devel
## Too long changelog entries
The following PRs have changelog entries that are too long to fit in a
single line. I'd expect authors to supply at changelog entries in
`DESCRIPTION:` lines that are at most 78 characters. If you want to
supply multi-line changelog items, you can have multiple lines that
start with `DESCRIPTION:` instead.
PR #6837 : fixes update propagation bug when
`citus_set_coordinator_host` is called more than once
PR #6738 : Identity column implementation refactorings
PR #6756 : Schedule parallel shard moves in background rebalancer by
removing task dependencies between shard moves across colocation groups.
PR #6793 : Add a GUC to disallow planning the queries that reference
non-colocated tables via router planner
PR #6726 : fix memory leak during altering distributed table with a lot
of partition and shards
PR #6722 : fix memory leak during distribution of a table with a lot of
partitions
PR #6693 : prevent memory leak during ConvertTable with a lot of
partitions
## Empty changelog entries.
The following PR had an empty `DESCRIPTION:` line. This generates an
empty changelog line that needs to be removed manually. Please either
provide a short entry, or remove `DESCRIPTION:` line completely.
PR #6810 : Make CDC decoder an independent extension
PR #6827 : Makefile changes to build CDC in builddir for pgoutput and
wal2json.
---------
Co-authored-by: Onur Tirtir <onurcantirtir@gmail.com>
We are handling colocation groups with shard group count less than the
worker node count, using a method different than the usual rebalancer.
See #6739
While making the decision of using this method or not, we should've
ignored the nodes that are marked `shouldhaveshards = false`. This PR
excludes those nodes when making the decision.
Adds a test such that:
coordinator: []
worker 1: [1_1, 1_2]
worker 2: [2_1, 2_2]
(rebalance)
coordinator: []
worker 1: [1_1, 2_1]
worker 2: [1_2, 2_2]
If we take the coordinator into account, the rebalancer considers the
first state as balanced and does nothing (because shard_count <
worker_count)
But with this pr, we ignore the coordinator because it's
shouldhaveshards = false
So the rebalancer distributes each colocation group to both workers
Also, fixes an unrelated flaky test in the same file
We need to break sequence dependency for a table while creating the
table during non-transactional metadata sync to ensure idempotency of
the creation of the table.
**Problem:**
When we send `SELECT
pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text)
FROM pg_dist_partition` to workers during the non-transactional sync,
table might not be in `pg_dist_partition` at worker, and sequence
dependency is not broken at the worker.
**Solution:**
We break sequence dependency via `SELECT
pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text)`
for each table while creating it at the workers. It is safe to send
since the udf is a no-op when there is no sequence dependency.
DESCRIPTION: Fixes a bug related to sequence idempotency at
non-transactional sync.
Fixes https://github.com/citusdata/citus/issues/6888.
When creating tags for backport releases, I realized that I missed one
changelog item. Adding it on the default branch in a commit. See #6885
for the relevant PR for the release branch.
There was a bug related to regex. We sometimes caught the wrong line
when the test name is also included in comments.
Example: We caught the wrong line as multi_metadata_sync is included in
the comment before the test line.
```
# ----------
# multi_metadata_sync tests the propagation of mx-related metadata changes to metadata workers
# multi_unsupported_worker_operations tests that unsupported operations error out on metadata workers
# ----------
test: multi_metadata_sync
```
Solution: Restrict regex rule better.
We had 10.1.5, 10.0.7, and 9.5.11 in the changelog, but those versions
are already used in enterprise repository. This commit skips those
versions and uses 10.1.6, 10.0.8, and 9.5.12 instead to prevent clashes.
We plan to have a series of backport releases. This PR contains separate
commits for each patch version for 11.2 to 9.5 major versions. We plan
to cherry pick each commit to relevant release branches and hence the
need to have separate commits for each version.
In #6814 we started using the Python test runner for upgrade tests in
run_test.py, instead of the Perl based one. This had a problem though,
not all tests in minimal_schedule can be run with the Python runner.
This adds a separate minimal schedule for the pg_upgrade tests which
doesn't include the tests that break with the Python runner.
This PR also fixes various other issues that came up while testing
the upgrade tests.
- Query generator is used to create queries, allowed by the grammar which is documented at `query_generator/query_gen.py` (currently contains only joins).
- This PR adds a CI test which utilizes the query generator to compare the results of generated queries that are executed on Citus tables and local (undistributed) tables. It fails if there is an unexpected error at results. The error can be related to Citus, the query generator, or even Postgres.
- The tool is configured by the file `query_generator/config/config.yaml`, which limits table counts at generated queries and sets many table related parameters (e.g. row count).
- Run time of the CI task can be configured from the config file. By default, we run 250 queries with maximum table count of 40 inside each query.
`PlaceHolderVar` is not relevant to be processed inside a restriction
clause. Otherwise, `pull_var_clause_default` would throw error. PG would
create the restriction to physical `Var` that `PlaceHolderVar` points to
anyway, so it is safe to skip this restriction.
DESCRIPTION: Fixes a bug related to WHERE clause list which contains
placeholder.
Fixes https://github.com/citusdata/citus/issues/6758
DESCRIPTION: Changes the regression test setups adding the coordinator
to metadata by default.
When creating a Citus cluster, coordinator can be added in metadata
explicitly by running `citus_set_coordinator_host ` function. Adding the
coordinator to metadata allows to create citus managed local tables.
Other Citus functionality is expected to be unaffected.
This change adds the coordinator to metadata by default when creating
test clusters in regression tests.
There are 3 ways to run commands in a sql file (or a schedule which is a
sequence of sql files) with Citus regression tests. Below is how this PR
adds the coordinator to metadata for each.
1. `make <schedule_name>`
Changed the sql files (sql/multi_cluster_management.sql and
sql/minimal_cluster_management.sql) which sets up the test clusters such
that they call `citus_set_coordinator_host`. This ensures any following
tests will have the coordinator in metadata by default.
2. `citus_tests/run_test.py <sql_file_name>`
Changed the python code that sets up the cluster to always call `
citus_set_coordinator_host`.
For the upgrade tests, a version check is included to make sure
`citus_set_coordinator_host` function is available for a given version.
3. ` make check-arbitrary-configs `
Changed the python code that sets up the cluster to always call
`citus_set_coordinator_host `.
#6864 will be used to track the remaining work which is to change the
tests where coordinator is added/removed as a node.
This PR updates the tenant stats implementation to set partitionKeyValue
and colocationId in ExecuteLocalTaskListExtended, in addition to
LocallyExecuteTaskPlan. This ensures that tenant stats can be properly
gathered regardless of the code path taken. The changes were initially
made while testing stored procedure calls for tenant stats.
.. rather than having it in user facing functions. That way, we
can use the same logic for creating Citus tables from other places
too.
This would be useful for creating tenant tables via a simple function
call in the utility hook, for schema-based sharding purposes.
DESCRIPTION: Fixes memory errors, caught by valgrind, of type
"conditional jump or move depends on uninitialized value"
When running Citus tests under Postgres with valgrind, the test cases
calling into `NonBlockingShardSplit` function produce valgrind errors of
type "conditional jump or move depends on uninitialized value".
The issue is caused by creating a HTAB in a wrong way. HASH_COMPARE flag
should have been used when creating a HTAB with user defined comparison
function. In the absence of HASH_COMPARE flag, HTAB falls back into
built-in string comparison function. However, valgrind somehow discovers
that the match function is not assigned to the user defined function as
intended.
Fixes#6835
Fixes the bug that causes updating the citus_stat_tenants periods
incorrectly.
`TimestampDifferenceExceeds` expects the difference in milliseconds but
it was microseconds, this is fixed.
`tenantStats->lastQueryTime` was updated during monitoring too, now it's
updated only when there are tenant queries.
The CDC decoder buillds different versions of CDC base decoders during
the build. Since the source files are copied to the temporay
directories, they come in git status for files to be added. So these
directories and a temporary CDC TAP test directory(tmpcheck) are added
to .gitignore file.
DESCRIPTION:
Makefile changes to build different versions of CDC decoder for different base decoders like pgoutput and wal2json with the same name and copy it to $packagelib/cdc_decoders dir. This helps the user to use logical replication slots normally with pgoutput without being aware of CDC decoder.
1) Changed src/backend/distributed/cdc/Makefile to setup a build directory
for CDC in build-cdc-$(DECODER) dir and copy the source files (.c.h and Makefile.decoder) to
the build dir and build it for each base decoder.
2) copy the pgoutput.so and wal2json.so into the above build dir and
install them in PG packagelibdir/citus_decoders directory.
3)Added a testcase 016_cdc_wal2json.pl for testing the wal2json decoder
using pg_recv_logical_changes function.
DESCRIPTION: Adds control for background task executors involving a node
### Background and motivation
Nonblocking concurrent task execution via background workers was
introduced in [#6459](https://github.com/citusdata/citus/pull/6459), and
concurrent shard moves in the background rebalancer were introduced in
[#6756](https://github.com/citusdata/citus/pull/6756) - with a hard
dependency that limits to 1 shard move per node. As we know, a shard
move consists of a shard moving from a source node to a target node. The
hard dependency was used because the background task runner didn't have
an option to limit the parallel shard moves per node.
With the motivation of controlling the number of concurrent shard
moves that involve a particular node, either as source or target, this
PR introduces a general new GUC
citus.max_background_task_executors_per_node to be used in the
background task runner infrastructure. So, why do we even want to
control and limit the concurrency? Well, it's all about resource
availability: because the moves involve the same nodes, extra
parallelism won’t make the rebalance complete faster if some resource is
already maxed out (usually cpu or disk). Or, if the cluster is being
used in a production setting, the moves might compete for resources with
production queries much more than if they had been executed
sequentially.
### How does it work?
A new column named nodes_involved is added to the catalog table that
keeps track of the scheduled background tasks,
pg_dist_background_task. It is of type integer[] - to store a list
of node ids. It is NULL by default - the column will be filled by the
rebalancer, but we may not care about the nodes involved in other uses
of the background task runner.
Table "pg_catalog.pg_dist_background_task"
Column | Type
============================================
job_id | bigint
task_id | bigint
owner | regrole
pid | integer
status | citus_task_status
command | text
retry_count | integer
not_before | timestamp with time zone
message | text
+nodes_involved | integer[]
A hashtable named ParallelTasksPerNode keeps track of the number of
parallel running background tasks per node. An entry in the hashtable is
as follows:
ParallelTasksPerNodeEntry
{
node_id // The node is used as the hash table key
counter // Number of concurrent background tasks that involve node node_id
// The counter limit is citus.max_background_task_executors_per_node
}
When the background task runner assigns a runnable task to a new
executor, it increments the counter for each of the nodes involved with
that runnable task. The limit of each counter is
citus.max_background_task_executors_per_node. If the limit is reached
for any of the nodes involved, this runnable task is skipped. And then,
later, when the running task finishes, the background task runner
decrements the counter for each of the nodes involved with the done
task. The following functions take care of these increment-decrement
steps:
IncrementParallelTaskCountForNodesInvolved(task)
DecrementParallelTaskCountForNodesInvolved(task)
citus.max_background_task_executors_per_node can be changed in the
fly. In the background rebalancer, we simply give {source_node,
target_node} as the nodesInvolved input to the
ScheduleBackgroundTask function. The rest is taken care of by the
general background task runner infrastructure explained above. Check
background_task_queue_monitor.sql and
background_rebalance_parallel.sql tests for detailed examples.
#### Note
This PR also adds a hard node dependency if a node is first being used
as a source for a move, and then later as a target. The reason this
should be a hard dependency is that the first move might make space for
the second move. So, we could run out of disk space (or at least
overload the node) if we move the second shard to it before the first
one is moved away.
Fixes https://github.com/citusdata/citus/issues/6716
DESCRIPTION: PR description that will go into the change log, up to 78
characters
---------
Co-authored-by: Hanefi Onaldi <Hanefi.Onaldi@microsoft.com>
Fixes flakiness in multi_metadata_sync test
https://app.circleci.com/pipelines/github/citusdata/citus/31863/workflows/ea937480-a4cc-4646-815c-bb2634361d98/jobs/1074457
```diff
SELECT
logicalrelid, repmodel
FROM
pg_dist_partition
WHERE
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass;
logicalrelid | repmodel
-----------------------------+----------
- mx_test_schema_1.mx_table_1 | s
mx_test_schema_2.mx_table_2 | s
+ mx_test_schema_1.mx_table_1 | s
(2 rows)
```
This is a simple issue of missing `ORDER BY` clauses. I went ahead and
added some other missing ones in the same file as well. Also, I replaced
existing `ORDER BY logicalrelid` with `ORDER BY logicalrelid::text`, in
order to compare names, not OIDs.
DESCRIPTION: Adds views that monitor statistics on tenant usages
This PR adds `citus_stats_tenants` view that monitors the tenants on the
cluster.
`citus_stats_tenants` shows the node id, colocation id, tenant
attribute, read count in this period and last period, and query count in
this period and last period of the tenant.
Tenant attribute currently is the tenant's distribution column value,
later when schema based sharding is introduced, this meaning might
change.
A period is a time bucket the queries are counted by. Read and query
counts for this period can increase until the current period ends. After
that those counts are moved to last period's counts, which cannot
change. The period length can be set using 'citus.stats_tenants_period'.
`SELECT` queries are counted as _read_ queries, `INSERT`, `UPDATE` and
`DELETE` queries are counted as _write_ queries. So in the view read
counts are `SELECT` counts and query counts are `SELECT`, `INSERT`,
`UPDATE` and `DELETE` count.
The data is stored in shared memory, in a struct named
`MultiTenantMonitor`.
`citus_stats_tenants` shows the data from local tenants.
`citus_stats_tenants` show up to `citus.stats_tenant_limit` number of
tenants.
The tenants are scored based on the number of queries they run and the
recency of those queries. Every query ran increases the score of tenant
by `ONE_QUERY_SCORE`, and after every period ends the scores are halved.
Halving is done lazily.
To retain information a longer the monitor keeps up to 3 times
`citus.stats_tenant_limit` tenants. When the tenant count hits `3 *
citus.stats_tenant_limit`, last `citus.stats_tenant_limit` tenants are
removed. To see all stored tenants you can use
`citus_stats_tenants(return_all_tenants := true)`
- [x] Create collector view that gets data from all nodes. #6761
- [x] Add monitoring log #6762
- [x] Create enable/disable GUC #6769
- [x] Parse the annotation string correctly #6796
- [x] Add local queries and prepared statements #6797
- [x] Rename to citus_stat_statements #6821
- [x] Run pgbench
- [x] Fix role permissions #6812
---------
Co-authored-by: Gokhan Gulbiz <ggulbiz@gmail.com>
Co-authored-by: Jelte Fennema <github-tech@jeltef.nl>