Commit Graph

103 Commits (faac9b968d7a0d031d85a7947592d2db3853a387)

Author SHA1 Message Date
Karina Litskevich faac9b968d Do not remove totaltime instrumentation from queryDesc 2025-06-23 18:55:42 +03:00
Naisila Puka 6bd3474804 Rename foreach_ macros to foreach_declared_ macros (#7700)
This is prep work for successful compilation with PG17

PG17added foreach_ptr, foreach_int and foreach_oid macros
Relevant PG commit
14dd0f27d7cd56ffae9ecdbe324965073d01a9ff

14dd0f27d7

We already have these macros, but they are different with the
PG17 ones because our macros take a DECLARED variable, whereas
the PG16 macros declare a locally-scoped loop variable themselves.

Hence I am renaming our macros to foreach_declared_

I am separating this into its own PR since it touches many files. The
main compilation PR is https://github.com/citusdata/citus/pull/7699
2025-03-12 11:01:49 +03:00
zhjwpku 51e607878b
remove a duplicate forward declaration and polish some comments (#7371)
remove a duplicate forward declaration and polish some comments

Signed-off-by: Zhao Junwang <zhjwpku@gmail.com>
2024-01-17 14:30:23 +00:00
Nils Dijk 0620c8f9a6
Sort includes (#7326)
This change adds a script to programatically group all includes in a
specific order. The script was used as a one time invocation to group
and sort all includes throught our formatted code. The grouping is as
follows:

 - System includes (eg. `#include<...>`)
 - Postgres.h (eg. `#include "postgres.h"`)
- Toplevel imports from postgres, not contained in a directory (eg.
`#include "miscadmin.h"`)
 - General postgres includes (eg . `#include "nodes/..."`)
- Toplevel citus includes, not contained in a directory (eg. `#include
"citus_verion.h"`)
 - Columnar includes (eg. `#include "columnar/..."`)
 - Distributed includes (eg. `#include "distributed/..."`)

Because it is quite hard to understand the difference between toplevel
citus includes and toplevel postgres includes it hardcodes the list of
toplevel citus includes. In the same manner it assumes anything not
prefixed with `columnar/` or `distributed/` as a postgres include.

The sorting/grouping is enforced by CI. Since we do so with our own
script there are not changes required in our uncrustify configuration.
2023-11-23 18:19:54 +01:00
Nils Dijk 0dac63afc0
move pg_version_constants.h to toplevel include (#7335)
In preparation of sorting and grouping all includes we wanted to move
this file to the toplevel includes for good grouping/sorting.
2023-11-09 15:09:39 +00:00
Naisila Puka 69af3e8509
Drop PG13 Support Phase 2 - Remove PG13 specific paths/tests (#7007)
This commit is the second and last phase of dropping PG13 support.

It consists of the following:

- Removes all PG_VERSION_13 & PG_VERSION_14 from codepaths
- Removes pg_version_compat entries and columnar_version_compat entries
specific for PG13
- Removes alternative pg13 test outputs 
- Removes PG13 normalize lines and fix the test outputs based on that

It is a continuation of 5bf163a27d
2023-06-21 14:18:23 +03:00
Marco Slot b09d239809 Propagate CREATE PUBLICATION statements 2023-03-29 00:59:12 +02:00
Jelte Fennema 81dcddd1ef
Actually skip constraint validation on shards after shard move (#6640)
DESCRIPTION: Fix foreign key validation skip at the end of shard move

In eadc88a we started completely skipping foreign key constraint
validation at the end of a non blocking shard move, instead of only for
foreign keys to reference tables. However, it turns out that this didn't
work at all because of a hard to notice bug: By resetting the
SkipConstraintValidation flag at the end of our utility hook, we
actually make the SET command that sets it a no-op.

This fixes that bug by removing the code that resets it. This is fine
because #6543 removed the only place where we set the flag in C code. So
the resetting of the flag has no purpose anymore. This PR also adds a
regression test, because it turned out we didn't have any otherwise we
would have caught that the feature was completely broken.

It also moves the constraint validation skipping to the utility hook.
The reason is that #6550 showed us that this is the better place to skip
it, because it will also skip the planning phase and not just the
execution.
2023-01-27 13:08:05 +01:00
Ahmet Gedemenli eadc88a800
Introduce GUC citus.skip_constraint_validation (#6281)
Introduces a new GUC named citus.skip_constraint_validation, which basically skips constraint validation when set to on.
For some several places that we hack to skip the foreign key validation phase, now we use this GUC.
2022-09-08 18:13:18 +03:00
Onder Kalaci 149771792b Remove useless version compats
most likely leftover from earlier versions
2022-07-29 10:31:55 +02:00
Marco Slot 7abcfac61f Add caching for functions that check the backend type 2022-05-20 19:02:37 +02:00
Marco Slot 09ec366ff5 Improve nested execution checks and add GUC to disable 2022-05-20 18:55:43 +02:00
Burak Velioglu 1460452442 Introduce CREATE/DROP VIEW
Adds support for propagating create/drop view commands and views to
worker node while scaling out the cluster. Since views are dropped while
converting the table type, metadata connection will be used while
propagating view commands to not switch to sequential mode.
2022-05-10 13:07:14 +03:00
Jeff Davis b6a5617ea8 PG15: handle pg_analyze_and_rewrite_* renaming.
From PG commit 791b1b71da.
2022-05-02 10:12:03 -07:00
Nils Dijk 8897361f95
Implement DOMAIN propagation for citus 2022-04-08 15:25:39 +02:00
Hanefi Onaldi b0eb685101
Add support for TEXT SEARCH DICTIONARY objects
TEXT SEARCH DICTIONARY objects depend on TEXT SEARCH TEMPLATE objects.
Since we do not yet support distributed TS TEMPLATE objects, we skip
dependency checks for text search templates, similar to what we do for
roles.

The user is expected to manually create the TEXT SEARCH TEMPLATE objects
before a) adding new nodes, b) creating TEXT SEARCH DICTIONARY objects.
2022-03-11 03:40:20 +03:00
Ahmet Gedemenli e1809af376 Propagate CREATE AGGREGATE commands 2022-03-02 10:52:43 +03:00
Nils Dijk 65bd540943
Feature: configure object propagation behaviour in transactions (#5724)
DESCRIPTION: Add GUC to control ddl creation behaviour in transactions

Historically we would _not_ propagate objects when we are in a transaction block. Creation of distributed tables would not always work in sequential mode, hence objects created in the same transaction as distributing a table that would use the just created object wouldn't work. The benefit was that the user could still benefit from parallelism.

Now that the creation of distributed tables is supported in sequential mode it would make sense for users to force transactional consistency of ddl commands for distributed tables. A transaction could switch more aggressively to sequential mode when creating new objects in a transaction.

We don't change the default behaviour just yet.

Also, many objects would not even propagate their creation when the transaction was already set to sequential, leaving the probability of a self deadlock. The new policy checks solve this discrepancy between objects as well.
2022-03-01 17:29:31 +03:00
Teja Mupparti 46fa47beea Force-delegated functions' distribution argument must be reset as soon as the routine completes execution,
and not wait until the top level Executor ends. This fixes issue #5687
2022-02-17 10:48:30 -08:00
Nils Dijk ea86f9f94e
Add support for TEXT SEARCH CONFIGURATION objects (#5685)
DESCRIPTION: Implement TEXT SEARCH CONFIGURATION propagation

The change adds support to Citus for propagating TEXT SEARCH CONFIGURATION objects. TSConfig objects cannot always be created in one create statement, and instead require a create statement followed by many alter statements to get turned into the object they should represent.

To support this we add functionality to the worker to create or replace objects based on a list of statements. When the lists of the local object and the remote object correspond 1:1 we skip the creation of the object and simply mark it distributed. This is especially important for TSConfig objects as initdb pre-populates databases with a dozen configurations (for many different languages).

When the user creates a new TSConfig based on the copy of an existing configuration there is no direct link to the object copied from. Since there is no link we can't simply rely on propagating the dependencies to the worker and send a qualified
2022-02-17 13:12:46 +01:00
Ahmet Gedemenli 0411a98c99
Refactor EnsureSequentialMode functions (#5704) 2022-02-14 18:38:21 +03:00
Teja Mupparti f31bce5b48 Fixes the issue seen in https://github.com/citusdata/citus-enterprise/issues/745
With this commit, rebalancer backends are identified by application_name = citus_rebalancer
and the regular internal backends are identified by application_name = citus_internal
2022-02-03 09:40:46 -08:00
Teja Mupparti 54862f8c22 (1) Functions will be delegated even when present in the scope of an explicit
BEGIN/COMMIT transaction block or in a UDF calling another UDF.
(2) Prohibit/Limit the delegated function not to do a 2PC (or any work on a
remote connection).
(3) Have a safety net to ensure the (2) i.e. we should block the connections
from the delegated procedure or make sure that no 2PC happens on the node.
(4) Such delegated functions are restricted to use only the distributed argument
value.

Note: To limit the scope of the project we are considering only Functions(not
procedures) for the initial work.

DESCRIPTION: Introduce a new flag "force_delegation" in create_distributed_function(),
which will allow a function to be delegated in an explicit transaction block.

Fixes #3265

Once the function is delegated to the worker, on that node during the planning

distributed_planner()
TryToDelegateFunctionCall()
CheckDelegatedFunctionExecution()
EnableInForceDelegatedFuncExecution()
Save the distribution argument (Constant)
ExecutorStart()
CitusBeginScan()
IsShardKeyValueAllowed()
Ensure to not use non-distribution argument.

ExecutorRun()
AdaptiveExecutor()
StartDistributedExecution()
EnsureNoRemoteExecutionFromWorkers()
Ensure all the shards are local to the node in the remoteTaskList.
NonPushableInsertSelectExecScan()
InitializeCopyShardState()
EnsureNoRemoteExecutionFromWorkers()
Ensure all the shards are local to the node in the placementList.

This also fixes a minor issue: Properly handle expressions+parameters in distribution arguments
2022-01-19 16:43:33 -08:00
Marco Slot ee3b50b026 Disallow remote execution from queries on shards 2022-01-07 17:46:21 +01:00
Önder Kalacı 8c0bc94b51
Enable replication factor > 1 in metadata syncing (#5392)
- [x] Add some more regression test coverage
- [x] Make sure returning works fine in case of
     local execution + remote execution
     (task->partiallyLocalOrRemote works as expected, already added tests)
- [x] Implement locking properly (and add isolation tests)
     - [x] We do #shardcount round-trips on `SerializeNonCommutativeWrites`.
           We made it a single round-trip.
- [x] Acquire locks for subselects on the workers & add isolation tests
- [x] Add a GUC to prevent modification from the workers, hence increase the
      coordinator-only throughput
       - The performance slightly drops (~%15), unless
         `citus.allow_modifications_from_workers_to_replicated_tables`
         is set to false
2021-11-15 15:10:18 +03:00
Philip Dubé cc50682158 Fix typos. Spurred spotting "connectios" in logs 2021-10-25 13:54:09 +00:00
Sait Talha Nisanci 0b67fcf81d Fix style 2021-09-03 16:09:59 +03:00
Halil Ozan Akgul 35cfa5d7b9 Introduces CopyFromState_compat macro
CopyState struct is divided into parts and one of them is CopyFromState
This macro uses the appropriate one for PG versions

Relevant PG commit:
c532d15dddff14b01fe9ef1d465013cb8ef186df
2021-09-03 15:27:24 +03:00
SaitTalhaNisanci 03832f353c Drop postgres 11 support 2021-03-25 09:20:28 +03:00
Hadi Moshayedi 7b74eca22d Support EXPLAIN EXECUTE ANALYZE. 2020-08-10 13:44:30 -07:00
Hanefi Onaldi 5be8287989
Fix comments of helper functions that set local config values (#4100) 2020-08-07 11:20:38 +03:00
Sait Talha Nisanci b641f63bfd Use CMDTAG_SELECT_COMPAT
CMDTAG_SELECT exists in PG12 hence defining a MACRO such as
CMDTAG_SELECT -> "SELECT" is not possible. I chose CMDTAG_SELECT_COMPAT
because with the COMPAT suffix it is explicit that it maps to different
things in different versions and also has a less chance of mapping
something irrevelant. For example if we used SELECT as a macro, then it
would map every SELECT to whatever it is mapping to, which might have
unexpected/undesired behaviour.
2020-08-04 15:38:13 +03:00
Sait Talha Nisanci 1112b254a7 adapt recently added code for pg13
This commit mostly adds pg_get_triggerdef_command to our ruleutils_13.
This doesn't add anything extra for ruleutils 13 so it is basically a copy
of the change on ruleutils_12
2020-08-04 15:18:27 +03:00
Sait Talha Nisanci 00e7386007 introduce PortalDefineQuerySelectCompat
PortalDefineQuery doesn't accept char* for command tag anymore with PG
>= 13. We are currently only using it with Select, therefore a Portal
define query compat for select is created.

Commit on PG side:
2f9661311b83dc481fc19f6e3bda015392010a40
2020-08-04 15:10:22 +03:00
Sait Talha Nisanci 62879ee8c1 introduce planner_compat and pg_plan_query_compat macros
As the new planner and pg_plan_query_compat methods expect the query
string as well, macros are defined to be compatible in different
versions of postgres.

Relevant commit on Postgres:
6aba63ef3e606db71beb596210dd95fa73c44ce2

Command on Postgres:
git log --all --grep="pg_plan_query"
2020-08-04 15:10:22 +03:00
SaitTalhaNisanci b3af63c8ce
Remove task tracker executor (#3850)
* use adaptive executor even if task-tracker is set

* Update check-multi-mx tests for adaptive executor

Basically repartition joins are enabled where necessary. For parallel
tests max adaptive executor pool size is decresed to 2, otherwise we
would get too many clients error.

* Update limit_intermediate_size test

It seems that when we use adaptive executor instead of task tracker, we
exceed the intermediate result size less in the test. Therefore updated
the tests accordingly.

* Update multi_router_planner

It seems that there is one problem with multi_router_planner when we use
adaptive executor, we should fix the following error:
+ERROR:  relation "authors_range_840010" does not exist
+CONTEXT:  while executing command on localhost:57637

* update repartition join tests for check-multi

* update isolation tests for repartitioning

* Error out if shard_replication_factor > 1 with repartitioning

As we are removing the task tracker, we cannot switch to it if
shard_replication_factor > 1. In that case, we simply error out.

* Remove MULTI_EXECUTOR_TASK_TRACKER

* Remove multi_task_tracker_executor

Some utility methods are moved to task_execution_utils.c.

* Remove task tracker protocol methods

* Remove task_tracker.c methods

* remove unused methods from multi_server_executor

* fix style

* remove task tracker specific tests from worker_schedule

* comment out task tracker udf calls in tests

We were using task tracker udfs to test permissions in
multi_multiuser.sql. We should find some other way to test them, then we
should remove the commented out task tracker calls.

* remove task tracker test from follower schedule

* remove task tracker tests from multi mx schedule

* Remove task-tracker specific functions from worker functions

* remove multi task tracker extra schedule

* Remove unused methods from multi physical planner

* remove task_executor_type related things in tests

* remove LoadTuplesIntoTupleStore

* Do initial cleanup for repartition leftovers

During startup, task tracker would call TrackerCleanupJobDirectories and
TrackerCleanupJobSchemas to clean up leftover directories and job
schemas. With adaptive executor, while doing repartitions it is possible
to leak these things as well. We don't retry cleanups, so it is possible
to have leftover in case of errors.

TrackerCleanupJobDirectories is renamed as
RepartitionCleanupJobDirectories since it is repartition specific now,
however TrackerCleanupJobSchemas cannot be used currently because it is
task tracker specific. The thing is that this function is a no-op
currently.

We should add cleaning up intermediate schemas to DoInitialCleanup
method when that problem is solved(We might want to solve it in this PR
as well)

* Revert "remove task tracker tests from multi mx schedule"

This reverts commit 03ecc0a681.

* update multi mx repartition parallel tests

* not error with task_tracker_conninfo_cache_invalidate

* not run 4 repartition queries in parallel

It seems that when we run 4 repartition queries in parallel we get too
many clients error on CI even though we don't get it locally. Our guess
is that, it is because we open/close many connections without doing some
work and postgres has some delay to close the connections. Hence even
though connections are removed from the pg_stat_activity, they might
still not be closed. If the above assumption is correct, it is unlikely
for it to happen in practice because:
- There is some network latency in clusters, so this leaves some times
for connections to be able to close
- Repartition joins return some data and that also leaves some time for
connections to be fully closed.

As we don't get this error in our local, we currently assume that it is
not a bug. Ideally this wouldn't happen when we get rid of the
task-tracker repartition methods because they don't do any pruning and
might be opening more connections than necessary.

If this still gives us "too many clients" error, we can try to increase
the max_connections in our test suite(which is 100 by default).

Also there are different places where this error is given in postgres,
but adding some backtrace it seems that we get this from
ProcessStartupPacket. The backtraces can be found in this link:
https://circleci.com/gh/citusdata/citus/138702

* Set distributePlan->relationIdList when it is needed

It seems that we were setting the distributedPlan->relationIdList after
JobExecutorType is called, which would choose task-tracker if
replication factor > 1 and there is a repartition query. However, it
uses relationIdList to decide if the query has a repartition query, and
since it was not set yet, it would always think it is not a repartition
query and would choose adaptive executor when it should choose
task-tracker.

* use adaptive executor even with shard_replication_factor > 1

It seems that we were already using adaptive executor when
replication_factor > 1. So this commit removes the check.

* remove multi_resowner.c and deprecate some settings

* remove TaskExecution related leftovers

* change deprecated API error message

* not recursively plan single relatition repartition subquery

* recursively plan single relation repartition subquery

* test depreceated task tracker functions

* fix overlapping shard intervals in range-distributed test

* fix error message for citus_metadata_container

* drop task-tracker deprecated functions

* put the implemantation back to worker_cleanup_job_schema_cachesince citus cloud uses it

* drop some functions, add downgrade script

Some deprecated functions are dropped.
Downgrade script is added.
Some gucs are deprecated.
A new guc for repartition joins bucket size is added.

* order by a test to fix flappiness
2020-07-18 13:11:36 +03:00
Marco Slot 634d6cf9d7
Improve performance of metadata cache (#3924)
#3866 removed the shard ID hash in metadata_cache.c to simplify cache management, 
but we observed a significant performance regression that was being masked by the
performance improvement provided by #3654 in our benchmarks, but #3654 only 
applies to specific workloads.

This PR brings back the shard ID cache as it existed before #3866 with some extra
 measures to handle invalidation. When we load a table entry, we overwrite 
ShardIdCacheEntry->tableEntry pointers for all the shards in that table, though 
it's possible that the table no longer contains the old shard ID or the table 
entry is never reloaded, which would leave a dangling pointer once the table 
entry is freed. To handle that case, we remove all shard ID cache entries that 
point exactly to that table entry when a table is freed (at the end of the 
transaction or any call to CitusTableCacheFlushInvalidatedEntries).

Co-authored-by: SaitTalhaNisanci <s.talhanisanci@gmail.com>
Co-authored-by: Marco Slot <marco.slot@gmail.com>
Co-authored-by: Jelte Fennema <github-tech@jeltef.nl>
2020-06-30 12:10:10 +02:00
Marco Slot 2a3234ca26 Rename masterQuery to combineQuery 2020-06-17 14:14:37 +02:00
Marco Slot d1bab78d79 Remove master from file hierarchy 2020-06-16 17:49:09 +02:00
Philip Dubé 39400319e6 Defer freeing CitusTableCacheEntry, as there were memory safety issues before
Shard id to index mapping stored in cache entry as there may now be multiple entries alive for a given relation

insert_select_executor: revert copying cache entry, which was a hack added to avoid memory safety issues
2020-06-15 16:20:50 +00:00
SaitTalhaNisanci 24dcb02bca
enable local table join with reference table (#3697)
* enable local table join with reference table

* test different cases with local table and reference join
2020-04-09 15:25:54 +03:00
SaitTalhaNisanci ba01f3457a
use macros for pg versions instead of hardcoded values (#3694)
3 Macros are defined for removing the hardcoded pg versions.
PG_VERSION_11, PG_VERSION_12 and PG_VERSION_13.
2020-04-01 17:01:52 +03:00
Onur Tirtir a14739f808
Local execution of ddl/drop/truncate commands (#3514)
* reimplement ExecuteUtilityTaskListWithoutResults for local utility command execution

* introduce new functions for local execution of utility commands

* change ErrorIfTransactionAccessedPlacementsLocally logic for local utility command execution

* enable local execution for TRUNCATE command on distributed & reference tables

* update existing tests for local utility command execution

* enable local execution for DDL commands on distributed & reference tables

* enable local execution for DROP command on distributed & reference tables

* add normalization rules for cascaded commands

* add new tests for local utility command execution
2020-03-13 15:39:32 +03:00
Philip Dubé 20abc4d2b5
Replace foreach with foreach_ptr/foreach_oid (#3544) 2020-02-27 16:54:49 +01:00
Nils Dijk a77ed9cd23
Refactor master query to be planned by postgres' planner (#3326)
DESCRIPTION: Replace the query planner for the coordinator part with the postgres planner

Closes #2761 

Citus had a simple rule based planner for the query executed on the query coordinator. This planner grew over time with the addigion of SQL support till it was getting close to the functionality of the postgres planner. Except the code was brittle and its complexity rose which made it hard to add new SQL support.

Given its resemblance with the postgres planner it was a long outstanding wish to replace our hand crafted planner with the well supported postgres planner. This patch replaces our planner with a call to postgres' planner.

Due to the functionality of the postgres planner we needed to support both projections and filters/quals on the citus custom scan node. When a sort operation is planned above the custom scan it might require fields to be reordered in the custom scan before returning the tuple (projection). The postgres planner assumes every custom scan node implements projections. Because we controlled the plan that was created we prevented reordering in the custom scan and never had implemented it before.

A same optimisation applies to having clauses that could have been where clauses. Instead of applying the filter as a having on the aggregate it will push it down into the plan which could reach a custom scan node.

For both filters and projections we have implemented them when tuples are read from the tuple store. If no projections or filters are required it will directly return the tuple from the tuple store. Otherwise it will loop tuples from the tuple store through the filter and projection until a tuple is found and returned.

Besides filters being pushed down a side effect of having quals that could have been a where clause is that a call to read intermediate result could be called before the first tuple is fetched from the custom scan. This failed because the intermediate result would only be pulled to the coordinator on the first tuple fetch. To overcome this problem we do run the distributed subplans now before we run the postgres executor. This ensures the intermediate result is present on the coordinator in time. We do account for total time instrumentation by removing the instrumentation before handing control to the psotgres executor and update the timings our self.

For future SQL support it is enough to create a valid query structure for the part of the query to be executed on the query coordinating node. As a utility we do serialise and print the query at debug level4 for engineers to inspect what kind of query is being planned on the query coordinator.
2020-02-25 14:39:56 +01:00
Nils Dijk bfc3d2eb90
make sure to correctly decrement ExecutorLevel (#3311)
DESCRIPTION: Fix counter that keeps track of internal depth in executor

While reviewing #3302 I ran into the `ExecutorLevel` variable which used a variable to keep the original value to restore on successful exit. I haven't explored the full space and if it is possible to get into an inconsistent state. However using `PG_TRY`/`PG_CATCH` seems generally more correct.

Given very bad things will happen if this level is not reset, I kept the failsafe of setting the variiable back to 0 on the `XactCallback` but I did add an assert to treat it as a developer bug.
2019-12-16 20:50:13 +01:00
Hadi Moshayedi 939d3c955b Don't plan function joins locally 2019-12-11 16:53:29 -08:00
Hadi Moshayedi e3e174f30f Fix the way we check for local/reference table joins in the executor 2019-12-11 12:50:20 -08:00
Hadi Moshayedi d28beb3711 Detect SQL UDF Calls. 2019-12-05 14:31:05 -08:00
Philip Dubé 5a17fd6d9d Test more reference/local cases, also ALTER ROLE
Test ALTER ROLE doesn't deadlock when coordinator added, or propagate from mx workers

Consolidate wait_until_metadata_sync & verify_metadata to multi_test_helpers
2019-12-03 22:23:14 +00:00