Each PostgreSQL backend starts with a predefined amount of stack and this stack
size can be increased if there is a need. However, stack size increase during
high memory load may cause unexpected crashes, because if there is not enough
memory for stack size increase, there is nothing to do for process apart from
crashing. An interesting thing is; the process would get OOM error instead of
crash, if the process had an explicit memory request (with palloc) for example.
However, in the case of stack size increase, there is no system call to get OOM
error, so the process simply crashes.
With this change, we are increasing the stack size explicitly by requesting extra
memory from the stack, so that, even if there is not memory, we can at least get
an OOM instead of a crash.
This commit by default enables hiding shard names on MX workers
by simple replacing `pg_table_is_visible()` calls with
`citus_table_is_visible()` calls on the MX worker nodes. The latter
function filters out tables that are known to be shards.
The main motivation of this change is a better UX. The functionality
can be opted out via a GUC.
We also added two views, namely citus_shards_on_worker and
citus_shard_indexes_on_worker such that users can query
them to see the shards and their corresponding indexes.
We also added debug messages such that the filtered tables can
be interactively seen by setting the level to DEBUG1.
When a hash distributed table have a foreign key to a reference
table, there are few restrictions we have to apply in order to
prevent distributed deadlocks or reading wrong results.
The necessity to apply the restrictions arise from cascading
nature of foreign keys. When a foreign key on a reference table
cascades to a distributed table, a single operation over a single
connection can acquire locks on multiple shards of the distributed
table. Thus, any parallel operation on that distributed table, in the
same transaction should not open parallel connections to the shards.
Otherwise, we'd either end-up with a self-distributed deadlock or
read wrong results.
As briefly described above, the restrictions that we apply is done
by tracking the distributed/reference relation accesses inside
transaction blocks, and act accordingly when necessary.
The two main rules are as follows:
- Whenever a parallel distributed relation access conflicts
with a consecutive reference relation access, Citus errors
out
- Whenever a reference relation access is followed by a
conflicting parallel relation access, the execution mode
is switched to sequential mode.
There are also some other notes to mention:
- If the user does SET LOCAL citus.multi_shard_modify_mode
TO 'sequential';, all the queries should simply work with
using one connection per worker and sequentially executing
the commands. That's obviously a slower approach than Citus'
usual parallel execution. However, we've at least have a way
to run all commands successfully.
- If an unrelated parallel query executed on any distributed
table, we cannot switch to sequential mode. Because, the essense
of sequential mode is using one connection per worker. However,
in the presence of a parallel connection, the connection manager
picks those connections to execute the commands. That contradicts
with our purpose, thus we error out.
- COPY to a distributed table cannot be executed in sequential mode.
Thus, if we switch to sequential mode and COPY is executed, the
operation fails and there is currently no way of implementing that.
Note that, when the local table is not empty and create_distributed_table
is used, citus uses COPY internally. Thus, in those cases,
create_distributed_table() will also fail.
- There is a GUC called citus.enforce_foreign_key_restrictions
to disable all the checks. We added that GUC since the restrictions
we apply is sometimes a bit more restrictive than its necessary.
The user might want to relax those. Similarly, if you don't have
CASCADEing reference tables, you might consider disabling all the
checks.
To support more flexible (i.e. not at compile-time) specification of
libpq connection parameters, this change adds a new GUC, node_conninfo,
which must be a space-separated string of key-value pairs suitable for
parsing by libpq's connection establishment methods.
To avoid rebuilding and parsing these values at connection time, this
change also adds a cache in front of the configuration params to permit
immediate use of any previously-calculated parameters.
* Change worker_hash_partition_table() such that the
divergence between Citus planner's hashing and
worker_hash_partition_table() becomes the same.
* Rename single partitioning to single range partitioning.
* Add single hash repartitioning. Basically, logical planner
treats single hash and range partitioning almost equally.
Physical planner, on the other hand, treats single hash and
dual hash repartitioning almost equally (except for JoinPruning).
* Add a new GUC to enable this feature
After this change all the logic related to shard data fetch logic
will be removed. Planner won't plan any ShardFetchTask anymore.
Shard fetch related steps in real time executor and task-tracker
executor have been removed.
This commit introduces a new GUC to limit the intermediate
result size which we handle when we use read_intermediate_result
function for CTEs and complex subqueries.
Adds ```citus.enable_statistics_collection``` GUC variable, which ```true``` by default, unless built without libcurl. If statistics collection is enabled, sends basic usage data to Citus servers every 24 hours.
The data that is collected consists of:
- Citus version
- OS name & release
- Hardware Id
- Number of tables, rounded to next power of 2
- Size of data, rounded to next power of 2
- Number of workers
We added a new GUC citus.log_distributed_deadlock_detection
which is off by default. When set to on, we log some debug messages
related to the distributed deadlock to the server logs.
With this commit, the maintenance deamon starts to check for
distributed deadlocks.
We also introduced a GUC variable (distributed_deadlock_detection_factor)
whose value is multiplied with Postgres' deadlock_timeout. Setting
it to -1 disables the distributed deadlock detection.
We send SIGINT to a backend that is cancelled due to a deadlock. That
approach ends up being a very confusing error message.
With this commit we intercept the error messages and show a more
meaningful error message to the user.
Now that we already have the necessary infrastructure for detecting
distributed deadlocks. Thus, we don't need enable_deadlock_prevention
which is purely intended for preventing some forms of distributed
deadlocks.
This GUC has two settings, 'always' and 'never'. When it's set to
'never' all behavior stays exactly as it was prior to this commit. When
it's set to 'always' only SELECT queries are allowed to run, and only
secondary nodes are used when processing those queries.
Add some helper functions:
- WorkerNodeIsSecondary(), checks the noderole of the worker node
- WorkerNodeIsReadable(), returns whether we're currently allowed to
read from this node
- ActiveReadableNodeList(), some functions (namely, the ones on the
SELECT path) don't require working with Primary Nodes. They should call
this function instead of ActivePrimaryNodeList(), because the latter
will error out in contexts where we're not allowed to write to nodes.
- ActiveReadableNodeCount(), like the above, replaces
ActivePrimaryNodeCount().
- EnsureModificationsCanRun(), error out if we're not currently allowed
to run queries which modify data. (Either we're in read-only mode or
use_secondary_nodes is set)
Some parts of the code were switched over to use readable nodes instead
of primary nodes:
- Deadlock detection
- DistributedTableSize,
- the router, real-time, and task tracker executors
- ShardPlacement resolution
maxTaskStringSize determines the size of worker query string.
It was originally hard coded to a specific value. This has caused
issues at some users. Since it determines initial shared memory
allocation, we did not want to set it to an arbitrary higher number.
Instead made it configurable.
This commit introduces a new GUC variable max_task_string_size
Changes in this variable requires restart to be in effect.
This commit adds distributed transaction id infrastructure in
the scope of distributed deadlock detection.
In general, the distributed transaction id consists of a tuple
in the form of: `(databaseId, initiatorNodeIdentifier, transactionId,
timestamp)`.
Briefly, we add a shared memory block on each node, which holds some
information per backend (i.e., an array `BackendData backends[MaxBackends]`).
Later, on each coordinated transaction, Citus sends
`SELECT assign_distributed_transaction_id()` right after `BEGIN`.
For that backend on the worker, the distributed transaction id is set to
the values assigned via the function call.
The aim of the above is to correlate the transactions on the coordinator
to the transactions on the worker nodes.
Adds support for PostgreSQL 10 by copying in the requisite ruleutils
and updating all API usages to conform with changes in PostgreSQL 10.
Most changes are fairly minor but they are numerous. One particular
obstacle was the change in \d behavior in PostgreSQL 10's psql; I had
to add SQL implementations (views, mostly) to mimic the pre-10 output.
In this PR, we aim to deduce whether each of the RTE_RELATION
is joined with at least on another RTE_RELATION on their partition keys. If each
RTE_RELATION follows the above rule, we can conclude that all RTE_RELATIONs are
joined on their partition keys.
In order to do that, we invented a new equivalence class namely:
AttributeEquivalenceClass. In very simple words, a AttributeEquivalenceClass is
identified by an unique id and consists of a list of AttributeEquivalenceMembers.
Each AttributeEquivalenceMember is designed to identify attributes uniquely within the
whole query. The necessity of this arise since varno attributes are defined within
a single level of a query. Instead, here we want to identify each RTE_RELATION uniquely
and try to find equality among each RTE_RELATION's partition key.
Whenever we find an equality clause A = B, where both A and B originates from
relation attributes (i.e., not random expressions), we create an
AttributeEquivalenceClass to record this knowledge. If we later find another
equivalence B = C, we create another AttributeEquivalenceClass. Finally, we can
apply transitity rules and generate a new AttributeEquivalenceClass which includes
A, B and C.
Note that equality among the members are identified by the varattno and rteIdentity.
Each equality among RTE_RELATION is saved using an AttributeEquivalenceClass where
each member attribute is identified by a AttributeEquivalenceMember. In the final
step, we try generate a common attribute equivalence class that holds as much as
AttributeEquivalenceMembers whose attributes are a partition keys.
With this change, we start to error out if loaded citus binaries does not match
the available major version or installed citus extension version. In this case
we force user to restart the server or run ALTER EXTENSION depending on the
situation
Custom Scan is a node in the planned statement which helps external providers
to abstract data scan not just for foreign data wrappers but also for regular
relations so you can benefit your version of caching or hardware optimizations.
This sounds like only an abstraction on the data scan layer, but we can use it
as an abstraction for our distributed queries. The only thing we need to do is
to find distributable parts of the query, plan for them and replace them with
a Citus Custom Scan. Then, whenever PostgreSQL hits this custom scan node in
its Vulcano style execution, it will call our callback functions which run
distributed plan and provides tuples to the upper node as it scans a regular
relation. This means fewer code changes, fewer bugs and more supported features
for us!
First, in the distributed query planner phase, we create a Custom Scan which
wraps the distributed plan. For real-time and task-tracker executors, we add
this custom plan under the master query plan. For router executor, we directly
pass the custom plan because there is not any master query. Then, we simply let
the PostgreSQL executor run this plan. When it hits the custom scan node, we
call the related executor parts for distributed plan, fill the tuple store in
the custom scan and return results to PostgreSQL executor in Vulcano style,
a tuple per XXX_ExecScan() call.
* Modify planner to utilize Custom Scan node.
* Create different scan methods for different executors.
* Use native PostgreSQL Explain for master part of queries.
This adds a replication_model GUC which is used as the replication
model for any new distributed table that is not a reference table.
With this change, tables with replication factor 1 are no longer
implicitly MX tables.
The GUC is similarly respected during empty shard creation for e.g.
existing append-partitioned tables. If the model is set to streaming
while replication factor is greater than one, table and shard creation
routines will error until this invalid combination is corrected.
Changing this parameter requires superuser permissions.
They make fixing explain for prepared statement harder, and they don't
really fit into EXPLAIN in the first place. Additionally they're
currently not exercised in any tests.
Currently there are several places in citus that map placements to
connections and that manage placement health. Centralize this
knowledge. Because of the centralized knowledge about which
connection has previously been used for which shard/placement, this
also provides the basis for relaxing restrictions around combining
various forms of DDL/DML.
Connections for a placement can now be acquired using
GetPlacementConnection(). If the connection is used for DML or DDL the
FOR_DDL/DML flags should be used respectively. If an individual
remote transaction fails (but the transaction on the master succeeds)
and FOR_DDL/DML have been specified, the placement is marked as
invalid, unless that'd mark all placements for a shard as invalid.
We used to disable router planner and executor
when task executor is set to task-tracker.
This change enables router planning and execution
at all times regardless of task execution mode.
We are introducing a hidden flag enable_router_execution
to enable/disable router execution. Its default value is
true. User may disable router planning by setting it to false.
One less place managing remote transactions. It also makes it fairly
easy to use 2PC for certain modifications (e.g. reference tables). Just
issue a CoordinatedTransactionUse2PC(). If every placement failure
should cause the whole transaction to abort, additionally mark the
relevant transactions as critical.
This includes basic infrastructure for logging of commands sent to
remote/worker nodes. Note that this has no effect as of yet, since no
callers are converted to the new infrastructure.
Connections are tracked and released by integrating into postgres'
transaction handling. That allows to to use connections without having
to resort to having to disable interrupts or using PG_TRY/CATCH blocks
to avoid leaking connections.
This is intended to eventually replace multi_client_executor.c and
connection_cache.c, and to provide the basis of a centralized
transaction management.
The newly introduced transaction hook should, in the future, be the only
one in citus, to allow for proper ordering between operations. For now
this central handler is responsible for releasing connections and
resetting XactModificationLevel after a transaction.
Adds support for PostgreSQL 9.6 by copying in the requisite ruleutils
file and refactoring the out/readfuncs code to flexibly support the
old-style copy/pasted out/readfuncs (prior to 9.6) or use extensible
node APIs (in 9.6 and higher).
Most version-specific code within this change is only needed to set new
fields in the AggRef nodes we build for aggregations. Version-specific
test output files were added in certain cases, though in most they were
not necessary. Each such file begins by e.g. printing the major version
in order to clarify its purpose.
The comment atop citus_nodes.h details how to add support for new nodes
for when that becomes necessary.
Related to #786
This change adds the `pg_dist_node` table that contains the information
about the workers in the cluster, replacing the previously used
`pg_worker_list.conf` file (or the one specified with `citus.worker_list_file`).
Upon update, `pg_worker_list.conf` file is read and `pg_dist_node` table is
populated with the file's content. After that, `pg_worker_list.conf` file
is renamed to `pg_worker_list.conf.obsolete`
For adding and removing nodes, the change also includes two new UDFs:
`master_add_node` and `master_remove_node`, which require superuser
permissions.
'citus.worker_list_file' guc is kept for update purposes but not used after the
update is finished.
I had changed these callbacks to use the same method I chose for the
router executor (for consistency), but as that method is flawed, we now
want to ensure we directly register them from PG_init as well.
Not entirely sure why we went with the shared memory hook approach, but
it causes problems (multiple registration) during crashes. Changing to
a simple direct registration call from PG_init.
Three changes here to get to true multi-statement, multi-relation DDL
transactions (same functionality pre-5.2, with benefits of atomicity):
1. Changed the multi-shard utility hook to always run (consistency
with router executor hook, removes ad-hoc "installed" boolean)
2. Change the global connection list in multi_shard_transaction to
instead be a hash; update related functions to operate on global
hash instead of local hash/global list
3. Remove check within DDL code to prevent subsequent DDL commands;
place unset/reset guard around call to ConnectToNode to permit
connecting to additional nodes after DDL transaction has begun
In addition, code has been added to raise an error if a ROLLBACK TO
SAVEPOINT is attempted (similar to router executor), and comprehensive
tests execute all multi-DDL scenarios (full success, user ROLLBACK, any
actual errors (say, duplicate index), partial failure (duplicate index
on one node but not others), partial COMMIT (one node fails), and 2PC
partial PREPARE (one node fails)). Interleavings with other commands
(DML, \copy) are similarly all covered.
We can now support richer set of queries in router planner.
This allow us to support CTEs, joins, window function, subqueries
if they are known to be executed at a single worker with a single
task (all tables are filtered down to a single shard and a single
worker contains all table shards referenced in the query).
Fixes : #501
Allows the use of modification commands (INSERT/UPDATE/DELETE) within
transaction blocks (delimited by BEGIN and ROLLBACK/COMMIT), so long as
all modifications hit a subset of nodes involved in the first such com-
mand in the transaction. This does not circumvent the requirement that
each individual modification command must still target a single shard.
For instance, after sending BEGIN, a user might INSERT some rows to a
shard replicated on two nodes. Subsequent modifications can hit other
shards, so long as they are on one or both of these nodes.
SAVEPOINTs are supported, though if the user actually attempts to send
a ROLLBACK command that specifies a SAVEPOINT they will receive an
ERROR at the end of the topmost transaction.
Placements are only marked inactive if at least one replica succeeds
in a transaction where others fail. Non-atomic behavior is possible if
the shard targeted by the initial modification within a transaction has
a higher replication factor than another shard within the same block
and a node with the latter shard has a failure during the COMMIT phase.
Other methods of denoting transaction blocks (multi-statement commands
sent all at once and functions written in e.g. PL/pgSQL or other such
languages) are not presently supported; their treatment remains the
same as before.
Fixes#513
This change modifies the DDL Propagation logic so that DDL queries
are propagated via 2-Phase Commit protocol. This way, failures during
the execution of distributed DDL commands will not leave the table in
an intermediate state and the pending prepared transactions can be
commited manually.
DDL commands are not allowed inside other transaction blocks or functions.
DDL commands are performed with 2PC regardless of the value of
`citus.multi_shard_commit_protocol` parameter.
The workflow of the successful case is this:
1. Open individual connections to all shard placements and send `BEGIN`
2. Send `SELECT worker_apply_shard_ddl_command(<shardId>, <DDL Command>)`
to all connections, one by one, in a serial manner.
3. Send `PREPARE TRANSCATION <transaction_id>` to all connections.
4. Sedn `COMMIT` to all connections.
Failure cases:
- If a worker problem occurs before sending of all DDL commands is finished, then
all changes are rolled back.
- If a worker problem occurs after all DDL commands are sent but not after
`PREPARE TRANSACTION` commands are finished, then all changes are rolled back.
However, if a worker node is failed, then the prepared transactions in that worker
should be rolled back manually.
- If a worker problem occurs during `COMMIT PREPARED` statements are being sent,
then the prepared transactions on the failed workers should be commited manually.
- If master fails before the first 'PREPARE TRANSACTION' is sent, then nothing is
changed on workers.
- If master fails during `PREPARE TRANSACTION` commands are being sent, then the
prepared transactions on workers should be rolled back manually.
- If master fails during `COMMIT PREPARED` or `ROLLBACK PREPARED` commands are being
sent, then the remaining prepared transactions on the workers should be handled manually.
This change also helps with #480, since failed DDL changes no longer mark
failed placements as inactive.
Now, master_create_empty_shard() will create shards according to the
value of citus.shard_placement_policy which also makes default round-robin
instead of random.
This change renames the distributed transaction manager parameter from
citus.copy_transaction_manager to citus.multi_shard_commit_protocol.
Distributed transaction manager has been used only by the COPY on hash
partitioned tables but it can be used by upcoming features so, we needed
to rename so that its name do not contain a reference to COPY.
The change also includes renames like transaction_manager_options to
commit_protocol_options and TRANSACTION_MANAGER_1PC to COMMIT_PROTOCOL_1PC.
With this change, declaration of MultiShardCommitProtocol (was
CopyTransactionManager) is moved from multi_copy.c to multi_transaction.c.
- non-router plannable queries can be executed
by router executor if they satisfy the criteria
- router executor is removed from configuration,
now task executor can not be set to router
- removed some tests that error out for router executor
Fixes#363
This change modifies the error message given when Citus is attempted
to be loaded other than shared_preload_libraries. Explanations have been
extended with that shared_preload_parameters parameter is in
postgresql.conf and citus should be at the beginning.
All citusdb references in
- extension, binary names
- file headers
- all configuration name prefixes
- error/warning messages
- some functions names
- regression tests
are changed to be citus.