Commit Graph

193 Commits (8976f245abc5465cabade8823f9c36deba2e442a)

Author SHA1 Message Date
Philip Dubé 4d9a733c2f Fix inserting multiple values with row expression partition column causing the insert to be ignored
Raise an error instead of silently inserting nothing if we hit this condition in the future
2020-01-15 21:10:50 +00:00
Philip Dubé 4b5d6c3ebe Rename RelayFileState to ShardState
Replace FILE_ prefix with SHARD_STATE_
2020-01-12 05:57:53 +00:00
Jelte Fennema 5b0baea72c Refactor distributed_planner for better understandability 2020-01-06 14:23:38 +01:00
Onder Kalaci 5a1e752726 Apply feedback - add fastPath field to plan 2020-01-06 12:42:43 +01:00
Onder Kalaci 13a9b55695 Skip expensive checks when fast-path query
The definition of fast-path query is very strict. So, we don't need
to do some extra checks.
2020-01-06 12:42:43 +01:00
Onder Kalaci 7f3ab7892d Skip shard pruning when possible
We're already traversing the queryTree and finding the distribution
key value, so pass it to the later stages of the planning.
2020-01-06 12:42:43 +01:00
Onder Kalaci ca293116fa Reduce calls to FastPathRouterQuery()
Before this commit, we called it twice durning planning. Instead,
we save the information and pass it.
2020-01-06 12:42:43 +01:00
SaitTalhaNisanci 420e21919b
refactor extract distributed insert values rte (#3287) 2019-12-12 23:47:44 +03:00
Marco Slot e7a8db5493 Fix issue with some zero-shard modifications 2019-12-12 07:19:10 +01:00
SaitTalhaNisanci 13204487e9
remove copyright years (#3286) 2019-12-11 21:14:08 +03:00
Marco Slot 486c620a3c Fix inserts into local tables with distributed subqueries 2019-12-10 10:17:18 +01:00
Philip Dubé fcf2fd819b Add distributioncolumncollation to to pg_dist_colocation
Use partition column's collation for range distributed tables
Don't allow non deterministic collations for hash distributed tables
CoPartitionedTables: don't compare unequal types
2019-12-09 19:51:40 +00:00
Marco Slot 6a9c0ea7fe Fix errors in DML with sublinks hidden by null expressions 2019-12-06 14:25:04 +01:00
Marco Slot bb3bc10f0c Fix segfault in column_to_column_name 2019-12-01 23:57:25 +01:00
Marco Slot 16d1ad3666 Remove distinction between SQL_TASK and ROUTER_TASK 2019-11-29 05:58:29 +01:00
SaitTalhaNisanci aeec3d1544
fix typo in dependent jobs and dependent task (#3244) 2019-11-28 23:47:28 +03:00
Philip Dubé 261a9de42d Fix typos:
VAR_SET_VALUE_KIND -> VAR_SET_VALUE kind
beginnig -> beginning
plannig -> planning
the the -> the
er then -> er than
2019-11-25 23:24:13 +00:00
Jelte Fennema 1d8dde232f
Automatically convert useless declarations using regex replace (#3181)
* Add declaration removal to CI

* Convert declarations
2019-11-21 13:47:29 +01:00
Onder Kalaci 90943a6ce6 Do not include coordinator shards when round-robin is selected
When the user picks "round-robin" policy, the aim is that the load
is distributed across nodes. However, for reference tables on the
coordinator, since local execution kicks in immediately, round-robin
is ignored.

With this change, we're excluding the placement on the coordinator.
Although the approach seems a little bit invasive because of
modifications in the placement list, that sounds acceptable.

We could have done this in some other ways such as:

1) Add a field to "Task->roundRobinPlacement" (or such), which is
updated as the first element after RoundRobinPolicy is applied.
During the execution, if that placement is local to the coordinator,
skip it and try the other remote placements.

2) On TaskAccessesLocalNode()@local_execution.c, check
task_assignment_policy, if round-robin selected and there is local
placement on the coordinator, skip it. However, task assignment is done
on planning, but this decision is happening on the execution, which
could create weird edge cases.
2019-11-15 06:03:32 -08:00
Hadi Moshayedi 15af1637aa Replicate reference tables to coordinator. 2019-11-15 05:50:19 -08:00
SaitTalhaNisanci b9b7fd7660
add IsLoggableLevel utility function (#3149)
* add IsLoggableLevel utility function

* add function comment for IsLoggableLevel

* put ApplyLogRedaction to logutils
2019-11-15 14:59:13 +03:00
Önder Kalacı 0b3d4e55d9
Local execution should not change hasReturning for distributed tables (#3160)
It looks like the logic to prevent RETURNING in reference tables to
have duplicate entries that comes from local and remote executions
leads to missing some tuples for distributed tables.

With this PR, we're ensuring to kick in the logic for reference tables
only.
2019-11-08 12:49:56 +01:00
Onur TIRTIR d3f68bf44f
Fix view is not distributed error when view is used in modify statements (#3104) 2019-11-01 16:34:01 +03:00
Onur TIRTIR d5f83dc110
Refactor range table walkers (#3109) 2019-10-16 01:20:49 +03:00
Philip Dubé 018ad1c58e pg12: version_compat.h, tuples, oids, misc 2019-08-22 18:57:23 +00:00
Philip Dubé 68c4b71f93 Fix up includes with pg12 changes 2019-08-22 18:56:21 +00:00
Philip Dubé b77c52f95b PlanRouterQuery: don't store list of list of shard intervals in relationShardList 2019-08-02 14:08:57 +00:00
Philip Dubé 0915027389 DistributedPlan: replace operation with modLevel
This causes no behaviorial changes, only organizes better to implement modifying CTEs

Also rename ExtactInsertRangeTableEntry to ExtractResultRelationRTE,
as the source of this function didn't match the documentation

Remove Task's upsertQuery in favor of ROW_MODIFY_NONCOMMUTATIVE

Split up AcquireExecutorShardLock into more internal functions

Tests: Normalize multi_reference_table multi_create_table_constraints
2019-07-16 13:58:18 -07:00
Önder Kalacı 40da78c6fd
Introduce the adaptive executor (#2798)
With this commit, we're introducing the Adaptive Executor. 


The commit message consists of two distinct sections. The first part explains
how the executor works. The second part consists of the commit messages of
the individual smaller commits that resulted in this commit. The readers
can search for the each of the smaller commit messages on 
https://github.com/citusdata/citus and can learn more about the history
of the change.

/*-------------------------------------------------------------------------
 *
 * adaptive_executor.c
 *
 * The adaptive executor executes a list of tasks (queries on shards) over
 * a connection pool per worker node. The results of the queries, if any,
 * are written to a tuple store.
 *
 * The concepts in the executor are modelled in a set of structs:
 *
 * - DistributedExecution:
 *     Execution of a Task list over a set of WorkerPools.
 * - WorkerPool
 *     Pool of WorkerSessions for the same worker which opportunistically
 *     executes "unassigned" tasks from a queue.
 * - WorkerSession:
 *     Connection to a worker that is used to execute "assigned" tasks
 *     from a queue and may execute unasssigned tasks from the WorkerPool.
 * - ShardCommandExecution:
 *     Execution of a Task across a list of placements.
 * - TaskPlacementExecution:
 *     Execution of a Task on a specific placement.
 *     Used in the WorkerPool and WorkerSession queues.
 *
 * Every connection pool (WorkerPool) and every connection (WorkerSession)
 * have a queue of tasks that are ready to execute (readyTaskQueue) and a
 * queue/set of pending tasks that may become ready later in the execution
 * (pendingTaskQueue). The tasks are wrapped in a ShardCommandExecution,
 * which keeps track of the state of execution and is referenced from a
 * TaskPlacementExecution, which is the data structure that is actually
 * added to the queues and describes the state of the execution of a task
 * on a particular worker node.
 *
 * When the task list is part of a bigger distributed transaction, the
 * shards that are accessed or modified by the task may have already been
 * accessed earlier in the transaction. We need to make sure we use the
 * same connection since it may hold relevant locks or have uncommitted
 * writes. In that case we "assign" the task to a connection by adding
 * it to the task queue of specific connection (in
 * AssignTasksToConnections). Otherwise we consider the task unassigned
 * and add it to the task queue of a worker pool, which means that it
 * can be executed over any connection in the pool.
 *
 * A task may be executed on multiple placements in case of a reference
 * table or a replicated distributed table. Depending on the type of
 * task, it may not be ready to be executed on a worker node immediately.
 * For instance, INSERTs on a reference table are executed serially across
 * placements to avoid deadlocks when concurrent INSERTs take conflicting
 * locks. At the beginning, only the "first" placement is ready to execute
 * and therefore added to the readyTaskQueue in the pool or connection.
 * The remaining placements are added to the pendingTaskQueue. Once
 * execution on the first placement is done the second placement moves
 * from pendingTaskQueue to readyTaskQueue. The same approach is used to
 * fail over read-only tasks to another placement.
 *
 * Once all the tasks are added to a queue, the main loop in
 * RunDistributedExecution repeatedly does the following:
 *
 * For each pool:
 * - ManageWorkPool evaluates whether to open additional connections
 *   based on the number unassigned tasks that are ready to execute
 *   and the targetPoolSize of the execution.
 *
 * Poll all connections:
 * - We use a WaitEventSet that contains all (non-failed) connections
 *   and is rebuilt whenever the set of active connections or any of
 *   their wait flags change.
 *
 *   We almost always check for WL_SOCKET_READABLE because a session
 *   can emit notices at any time during execution, but it will only
 *   wake up WaitEventSetWait when there are actual bytes to read.
 *
 *   We check for WL_SOCKET_WRITEABLE just after sending bytes in case
 *   there is not enough space in the TCP buffer. Since a socket is
 *   almost always writable we also use WL_SOCKET_WRITEABLE as a
 *   mechanism to wake up WaitEventSetWait for non-I/O events, e.g.
 *   when a task moves from pending to ready.
 *
 * For each connection that is ready:
 * - ConnectionStateMachine handles connection establishment and failure
 *   as well as command execution via TransactionStateMachine.
 *
 * When a connection is ready to execute a new task, it first checks its
 * own readyTaskQueue and otherwise takes a task from the worker pool's
 * readyTaskQueue (on a first-come-first-serve basis).
 *
 * In cases where the tasks finish quickly (e.g. <1ms), a single
 * connection will often be sufficient to finish all tasks. It is
 * therefore not necessary that all connections are established
 * successfully or open a transaction (which may be blocked by an
 * intermediate pgbouncer in transaction pooling mode). It is therefore
 * essential that we take a task from the queue only after opening a
 * transaction block.
 *
 * When a command on a worker finishes or the connection is lost, we call
 * PlacementExecutionDone, which then updates the state of the task
 * based on whether we need to run it on other placements. When a
 * connection fails or all connections to a worker fail, we also call
 * PlacementExecutionDone for all queued tasks to try the next placement
 * and, if necessary, mark shard placements as inactive. If a task fails
 * to execute on all placements, the execution fails and the distributed
 * transaction rolls back.
 *
 * For multi-row INSERTs, tasks are executed sequentially by
 * SequentialRunDistributedExecution instead of in parallel, which allows
 * a high degree of concurrency without high risk of deadlocks.
 * Conversely, multi-row UPDATE/DELETE/DDL commands take aggressive locks
 * which forbids concurrency, but allows parallelism without high risk
 * of deadlocks. Note that this is unrelated to SEQUENTIAL_CONNECTION,
 * which indicates that we should use at most one connection per node, but
 * can run tasks in parallel across nodes. This is used when there are
 * writes to a reference table that has foreign keys from a distributed
 * table.
 *
 * Execution finishes when all tasks are done, the query errors out, or
 * the user cancels the query.
 *
 *-------------------------------------------------------------------------
 */



All the commits involved here:
* Initial unified executor prototype

* Latest changes

* Fix rebase conflicts to master branch

* Add missing variable for assertion

* Ensure that master_modify_multiple_shards() returns the affectedTupleCount

* Adjust intermediate result sizes

The real-time executor uses COPY command to get the results
from the worker nodes. Unified executor avoids that which
results in less data transfer. Simply adjust the tests to lower
sizes.

* Force one connection per placement (or co-located placements) when requested

The existing executors (real-time and router) always open 1 connection per
placement when parallel execution is requested.

That might be useful under certain circumstances:

(a) User wants to utilize as much as CPUs on the workers per
distributed query
(b) User has a transaction block which involves COPY command

Also, lots of regression tests rely on this execution semantics.
So, we'd enable few of the tests with this change as well.

* For parameters to be resolved before using them

For the details, see PostgreSQL's copyParamList()

* Unified executor sorts the returning output

* Ensure that unified executor doesn't ignore sequential execution of DDLJob's

Certain DDL commands, mainly creating foreign keys to reference tables,
should be executed sequentially. Otherwise, we'd end up with a self
distributed deadlock.

To overcome this situaiton, we set a flag `DDLJob->executeSequentially`
and execute it sequentially. Note that we have to do this because
the command might not be called within a transaction block, and
we cannot call `SetLocalMultiShardModifyModeToSequential()`.

This fixes at least two test: multi_insert_select_on_conflit.sql and
multi_foreign_key.sql

Also, I wouldn't mind scattering local `targetPoolSize` variables within
the code. The reason is that we'll soon have a GUC (or a global
variable based on a GUC) that'd set the pool size. In that case, we'd
simply replace `targetPoolSize` with the global variables.

* Fix 2PC conditions for DDL tasks

* Improve closing connections that are not fully established in unified execution

* Support foreign keys to reference tables in unified executor

The idea for supporting foreign keys to reference tables is simple:
Keep track of the relation accesses within a transaction block.
    - If a parallel access happens on a distributed table which
      has a foreign key to a reference table, one cannot modify
      the reference table in the same transaction. Otherwise,
      we're very likely to end-up with a self-distributed deadlock.
    - If an access to a reference table happens, and then a parallel
      access to a distributed table (which has a fkey to the reference
      table) happens, we switch to sequential mode.

Unified executor misses the function calls that marks the relation
accesses during the execution. Thus, simply add the necessary calls
and let the logic kick in.

* Make sure to close the failed connections after the execution

* Improve comments

* Fix savepoints in unified executor.

* Rebuild the WaitEventSet only when necessary

* Unclaim connections on all errors.

* Improve failure handling for unified executor

   - Implement the notion of errorOnAnyFailure. This is similar to
     Critical Connections that the connection managament APIs provide
   - If the nodes inside a modifying transaction expand, activate 2PC
   - Fix few bugs related to wait event sets
   - Mark placement INACTIVE during the execution as much as possible
     as opposed to we do in the COMMIT handler
   - Fix few bugs related to scheduling next placement executions
   - Improve decision on when to use 2PC

Improve the logic to start a transaction block for distributed transactions

- Make sure that only reference table modifications are always
  executed with distributed transactions
- Make sure that stored procedures and functions are executed
  with distributed transactions

* Move waitEventSet to DistributedExecution

This could also be local to RunDistributedExecution(), but in that case
we had to mark it as "volatile" to avoid PG_TRY()/PG_CATCH() issues, and
cast it to non-volatile when doing WaitEventSetFree(). We thought that
would make code a bit harder to read than making this non-local, so we
move it here. See comments for PG_TRY() in postgres/src/include/elog.h
and "man 3 siglongjmp" for more context.

* Fix multi_insert_select test outputs

Two things:
   1) One complex transaction block is now supported. Simply update
      the test output
   2) Due to dynamic nature of the unified executor, the orders of
      the errors coming from the shards might change (e.g., all of
      the queries on the shards would fail, but which one appears
      on the error message?). To fix that, we simply added it to
      our shardId normalization tool which happens just before diff.

* Fix subeury_and_cte test

The error message is updated from:
	failed to execute task
To:
        more than one row returned by a subquery or an expression

which is a lot clearer to the user.

* Fix intermediate_results test outputs

Simply update the error message from:
	could not receive query results
to
	result "squares" does not exist

which makes a lot more sense.

* Fix multi_function_in_join test

The error messages update from:
     Failed to execute task XXX
To:
     function f(..) does not exist

* Fix multi_query_directory_cleanup test

The unified executor does not create any intermediate files.

* Fix with_transactions test

A test case that just started to work fine

* Fix multi_router_planner test outputs

The error message is update from:
	Could not receive query results
To:
	Relation does not exists

which is a lot more clearer for the users

* Fix multi_router_planner_fast_path test

The error message is update from:
	Could not receive query results
To:
	Relation does not exists

which is a lot more clearer for the users

* Fix isolation_copy_placement_vs_modification by disabling select_opens_transaction_block

* Fix ordering in isolation_multi_shard_modify_vs_all

* Add executor locks to unified executor

* Make sure to allocate enought WaitEvents

The previous code was missing the waitEvents for the latch and
postmaster death.

* Fix rebase conflicts for master rebase

* Make sure that TRUNCATE relies on unified executor

* Implement true sequential execution for multi-row INSERTS

Execute the individual tasks executed one by one. Note that this is different than
MultiShardConnectionType == SEQUENTIAL_CONNECTION case (e.g., sequential execution
mode). In that case, running the tasks across the nodes in parallel is acceptable
and implemented in that way.

However, the executions that are qualified here would perform poorly if the
tasks across the workers are executed in parallel. We currently qualify only
one class of distributed queries here, multi-row INSERTs. If we do not enforce
true sequential execution, concurrent multi-row upserts could easily form
a distributed deadlock when the upserts touch the same rows.

* Remove SESSION_LIFESPAN flag in unified_executor

* Apply failure test updates

We've changed the failure behaviour a bit, and also the error messages
that show up to the user. This PR covers majority of the updates.

* Unified executor honors citus.node_connection_timeout

With this commit, unified executor errors out if even
a single connection cannot be established within
citus.node_connection_timeout.

And, as a side effect this fixes failure_connection_establishment
test.

* Properly increment/decrement pool size variables

Before this commit, the idle and active connection
counts were not properly calculated.

* insert_select_executor goes through unified executor.

* Add missing file for task tracker

* Modify ExecuteTaskListExtended()'s signature

* Sort output of INSERT ... SELECT ... RETURNING

* Take partition locks correctly in unified executor

* Alternative implementation for force_max_query_parallelization

* Fix compile warnings in unified executor

* Fix style issues

* Decrement idleConnectionCount when idle connection is lost

* Always rebuild the wait event sets

In the previous implementation, on waitFlag changes, we were only
modifying the wait events. However, we've realized that it might
be an over optimization since (a) we couldn't see any performance
benefits (b) we see some errors on failures and because of (a)
we prefer to disable it now.

* Make sure to allocate enough sized waitEventSet

With multi-row INSERTs, we might have more sessions than
task*workerCount after few calls of RunDistributedExecution()
because the previous sessions would also be alive.

Instead, re-allocate events when the connectino set changes.

* Implement SELECT FOR UPDATE on reference tables

On master branch, we do two extra things on SELECT FOR UPDATE
queries on reference tables:
   - Acquire executor locks
   - Execute the query on all replicas

With this commit, we're implementing the same logic on the
new executor.

* SELECT FOR UPDATE opens transaction block even if SelectOpensTransactionBlock disabled

Otherwise, users would be very confused and their logic is very likely
to break.

* Fix build error

* Fix the newConnectionCount calculation in ManageWorkerPool

* Fix rebase conflicts

* Fix minor test output differences

* Fix citus indent

* Remove duplicate sorts that is added with rebase

* Create distributed table via executor

* Fix wait flags in CheckConnectionReady

* failure_savepoints output for unified executor.

* failure_vacuum output (pg 10) for unified executor.

* Fix WaitEventSetWait timeout in unified executor

* Stabilize failure_truncate test output

* Add an ORDER BY to multi_upsert

* Fix regression test outputs after rebase to master

* Add executor.c comment

* Rename executor.c to adaptive_executor.c

* Do not schedule tasks if the failed placement is not ready to execute

Before the commit, we were blindly scheduling the next placement executions
even if the failed placement is not on the ready queue. Now, we're ensuring
that if failed placement execution is on a failed pool or session where the
execution is on the pendingQueue, we do not schedule the next task. Because
the other placement execution should be already running.

* Implement a proper custom scan node for adaptive executor

- Switch between the executors, add GUC to set the pool size
- Add non-adaptive regression test suites
- Enable CIRCLE CI for non-adaptive tests
- Adjust test output files

* Add slow start interval to the executor

* Expose max_cached_connection_per_worker to user

* Do not start slow when there are cached connections

* Consider ExecutorSlowStartInterval in NextEventTimeout

* Fix memory issues with ReceiveResults().

* Disable executor via TaskExecutorType

* Make sure to execute the tests with the other executor

* Use task_executor_type to enable-disable adaptive executor

* Remove useless code

* Adjust the regression tests

* Add slow start regression test

* Rebase to master

* Fix test failures in adaptive executor.

* Rebase to master - 2

* Improve comments & debug messages

* Set force_max_query_parallelization in isolation_citus_dist_activity

* Force max parallelization for creating shards when asked to use exclusive connection.

* Adjust the default pool size

* Expand description of max_adaptive_executor_pool_size GUC

* Update warnings in FinishRemoteTransactionCommit()

* Improve session clean up at the end of execution

Explicitly list all the states that the execution might end,
otherwise warn.

* Remove MULTI_CONNECTION_WAIT_RETRY which is not used at all

* Add more ORDER BYs to multi_mx_partitioning
2019-06-28 14:04:40 +02:00
Philip Dubé db7fdb1854 Router planner: bail on volatile functions in CTEs 2019-06-26 10:32:01 +02:00
Philip Dubé 5c62f9935a Router planner: reject SELECT FOR UPDATE ctes 2019-06-26 10:32:01 +02:00
Philip Dubé 77efec04a0 Router Planner: accept SELECT_CMD ctes in modification queries 2019-06-26 10:32:01 +02:00
Philip Dubé 84fe626378 multi_router_planner: refactor error propagation 2019-06-26 10:32:01 +02:00
Hanefi Onaldi 4d737177e6
Remove redundant active placement filters and unneded sort operations
If a query is router executable, it hits a single shard and therefore has a
single task associated with it. Therefore there is no need to sort the task list
that has a single element.

Also we already have a list of active shard placements, sending it in param
and reuse it.
2019-05-24 14:16:50 +03:00
Hanefi Onaldi 4030d603eb
Merge pull request #2691 from citusdata/update_changelog
Add 8.1.2 and 8.2.1 changelog entries
2019-05-15 09:18:58 +03:00
Jason Petersen 71d5d1c865 Enable variable shadowing warnings; fix all
Rather than wait for another place like the previous commit to bite us,
I think we should turn on this warning.
2019-04-30 13:24:25 -06:00
Marco Slot e8152d9b6d Only look in top-level rtable in ExtractFirstDistributedTableId 2019-03-20 12:14:46 +03:00
Marco Slot 0ea4e52df5 Add nodeId to shardPlacements and use it for shard placement comparisons
Before this commit, shardPlacements were identified with shardId, nodeName
and nodeport. Instead of using nodeName and nodePort, we now use nodeId
since it apparently has performance benefits in several places in the
code.
2019-03-20 12:14:46 +03:00
Onder Kalaci e521e7e39c Apply feedback 2019-02-22 18:14:30 +03:00
Onder Kalaci f144bb4911 Introduce fast path router planning
In this context, we define "Fast Path Planning for SELECT" as trivial
queries where Citus can skip relying on the standard_planner() and
handle all the planning.

For router planner, standard_planner() is mostly important to generate
the necessary restriction information. Later, the restriction information
generated by the standard_planner is used to decide whether all the shards
that a distributed query touches reside on a single worker node. However,
standard_planner() does a lot of extra things such as cost estimation and
execution path generations which are completely unnecessary in the context
of distributed planning.

There are certain types of queries where Citus could skip relying on
standard_planner() to generate the restriction information. For queries
in the following format, Citus does not need any information that the
standard_planner() generates:

  SELECT ... FROM single_table WHERE distribution_key = X;  or
  DELETE FROM single_table WHERE distribution_key = X; or
  UPDATE single_table SET value_1 = value_2 + 1 WHERE distribution_key = X;

Note that the queries might not be as simple as the above such that
GROUP BY, WINDOW FUNCIONS, ORDER BY or HAVING etc. are all acceptable. The
only rule is that the query is on a single distributed (or reference) table
and there is a "distribution_key = X;" in the WHERE clause. With that, we
could use to decide the shard that a distributed query touches reside on
a worker node.
2019-02-21 13:27:01 +03:00
Onder Kalaci ec67381ba2 Queries with only intermediate results do not rely on task assignment policy
Previously we allowed task assignment policy to have affect on router queries
with only intermediate results. However, that is erroneous since the code-path
that assigns placements relies on shardIds and placements, which doesn't exists
for intermediate results.

With this commit, we do not apply task assignment policies when a router query
hits only intermediate results.
2019-01-28 17:59:17 +03:00
Jason Petersen 339e6e661e
Remove 9.6 (#2554)
Removes support and code for PostgreSQL 9.6

cr: @velioglu
2019-01-16 13:11:24 -07:00
Marco Slot 8893cc141d Support INSERT...SELECT with ON CONFLICT or RETURNING via coordinator
Before this commit, Citus supported INSERT...SELECT queries with
ON CONFLICT or RETURNING clauses only for pushdownable ones, since
queries supported via coordinator were utilizing COPY infrastructure
of PG to send selected tuples to the target worker nodes.

After this PR, INSERT...SELECT queries with ON CONFLICT or RETURNING
clauses will be performed in two phases via coordinator. In the first
phase selected tuples will be saved to the intermediate table which
is colocated with target table of the INSERT...SELECT query. Note that,
a utility function to save results to the colocated intermediate result
also implemented as a part of this commit. In the second phase, INSERT..
SELECT query is directly run on the worker node using the intermediate
table as the source table.
2018-11-30 15:29:12 +03:00
Nils Dijk f9520be011
Round robin queries to reference tables with task_assignment_policy set to `round-robin` (#2472)
Description: Support round-robin `task_assignment_policy` for queries to reference tables.

This PR allows users to query multiple placements of shards in a round robin fashion. When `citus.task_assignment_policy` is set to `'round-robin'` the planner will use a round robin scheduling feature when multiple shard placements are available.

The primary use-case is spreading the load of reference table queries to all the nodes in the cluster instead of hammering only the first placement of the reference table. Since reference tables share the same path for selecting the shards with single shard queries that have multiple placements (`citus.shard_replication_factor > 1`) this setting also allows users to spread the query load on these shards.

For modifying queries we do not apply a round-robin strategy. This would be negated by an extra reordering step in the executor for such queries where a `first-replica` strategy is enforced.
2018-11-15 15:11:15 +01:00
Onder Kalaci c1b5a04f6e Allow partitioned tables with replication factor > 1
With this commit, we all partitioned distributed tables with
replication factor > 1. However, we also have many restrictions.

In summary, we disallow all kinds of modifications (including DDLs)
on the partition tables. Instead, the user is allowed to run the
modifications over the parent table.

The necessity for such a restriction have two aspects:
   - We need to acquire shard resource locks appropriately
   - We need to handle marking partitions INVALID in case
     of any failures. Note that, in theory, the parent table
     should also become INVALID, which is too aggressive.
2018-09-21 14:40:41 +03:00
velioglu bd30e3e908 Add support for writing to reference tables from MX nodes 2018-08-27 18:15:04 +03:00
Onder Kalaci 910ea392f5 Prevent multiple placements of a single shard to lead huge memory allocations 2018-08-22 19:25:01 +03:00
Nils Dijk 6a15e1c9fc
extract ErrorIfOnConflictNotSupported function for reuse 2018-07-23 12:20:10 +02:00
Murat Tuncer e532755a6e Fix bug in partition column extraction
added strip_implicit_coercion prior to
checking if the expression is Const.
This is important to find values for types
like bigint.
2018-07-02 18:08:16 +03:00
Murat Tuncer 4d35b92016 Add groundwork for citus_stat_statements api 2018-06-27 14:20:03 +03:00
velioglu 53b2e81d01 Adds SELECT ... FOR UPDATE support for router plannable queries 2018-06-18 13:55:17 +03:00
Marco Slot fd4ff29f2f Add a debug message with distribution column value 2018-06-05 15:09:17 +03:00
velioglu caa27161ca Check volatile functions in modify queries 2018-05-08 11:16:40 +03:00
Marco Slot 90cdfff602 Implement recursive planning for DML statements 2018-05-03 14:42:28 +02:00
velioglu 32bcd610c1 Support modify queries with multiple tables
With this commit we begin to support modify queries with multiple
tables if these queries are pushdownable.
2018-05-02 16:22:26 +03:00
Marco Slot ee132c5ead Prune shards once per relation in subquery pushdown 2018-04-10 20:33:07 +02:00
velioglu 698d585fb5 Remove broadcast join logic
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.
2018-03-30 11:45:19 +03:00
Onder Kalaci 40b898b59f Improve error messages for INSERT queries that have subqueries 2018-03-05 14:46:47 +02:00
Metin Doslu bcf660475a Add support for modifying CTEs 2018-02-27 15:08:32 +02:00
Marco Slot 2e2b4e81fa Add support for CTEs in distributed queries 2017-12-14 09:32:55 +01:00
Marco Slot 60a1e31671 Allow queries with local tables in NeedsDistributedPlanning 2017-12-07 16:20:23 +01:00
Marco Slot d8fea4efb8 Revert "Allow queries with local tables in NeedsDistributedPlanning"
This reverts commit d2bac081e8.
2017-12-07 11:19:11 +01:00
Marco Slot d2bac081e8 Allow queries with local tables in NeedsDistributedPlanning 2017-12-07 11:02:16 +01:00
Marco Slot 7ea718fd8d Round-robin over worker nodes for 0-shard router queries 2017-11-29 15:52:22 +01:00
Marco Slot 6ba3f42d23 Rename MultiPlan to DistributedPlan 2017-11-22 09:36:24 +01:00
velioglu 0b5db5d826 Support multi shard update/delete queries 2017-10-25 15:52:38 +03:00
Marco Slot 0aadbb1760 Convert multi-row INSERT target list to Vars 2017-08-25 10:55:56 +02:00
Marco Slot ae00795dab Allow default columns in multi-row INSERTs 2017-08-25 10:55:56 +02:00
Marco Slot c97692f382 Fix multi-row INSERT with RETURNING on reference tables 2017-08-24 10:42:12 +02:00
velioglu ceba81ce35 Move physical planner checks to logical planner 2017-08-11 10:09:47 +03:00
velioglu c4e3b8b5e1 Add planner changes and tests for subquery on reference tables 2017-08-11 10:09:47 +03:00
Brian Cloutier 9d93fb5551 Create citus.use_secondary_nodes GUC
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
2017-08-10 17:37:17 +03:00
Jason Petersen dee66e3959
Final review feedback 2017-08-10 01:10:09 -07:00
Jason Petersen 6a35c2937c
Enable multi-row INSERTs
This is a pretty substantial refactoring of the existing modify path
within the router executor and planner. In particular, we now hunt for
all VALUES range table entries in INSERT statements and group the rows
contained therein by shard identifier. These rows are stashed away for
later in "ModifyRoute" elements. During deparse, the appropriate RTE
is extracted from the Query and its values list is replaced by these
rows before any SQL is generated.

In this way, we can create multiple Tasks, but only one per shard, to
piecemeal execute a multi-row INSERT. The execution of jobs containing
such tasks now exclusively go through the "multi-router executor" which
was previously used for e.g. INSERT INTO ... SELECT.

By piggybacking onto that executor, we participate in ongoing trans-
actions, get rollback-ability, etc. In short order, the only remaining
use of the "single modify" router executor will be for bare single-
row INSERT statements (i.e. those not in a transaction).

This change appropriately handles deferred pruning as well as master-
evaluated functions.
2017-08-10 00:32:46 -07:00
Metin Doslu b8a9e7c1bf Add support for UPDATE/DELETE with subqueries 2017-08-08 21:35:08 +03:00
Marco Slot aa7ca81548 Execute UPDATE/DELETE statements with 0 shards 2017-08-07 15:36:58 +02:00
Brian Cloutier ec99f8f983 Add nodeRole column
- master_add_node enforces that there is only one primary per group
- there's also a trigger on pg_dist_node to prevent multiple primaries
  per group
- functions in metadata cache only return primary nodes
- Rename ActiveWorkerNodeList -> ActivePrimaryNodeList
- Rename WorkerGetLive{Node->Group}Count()
- Refactor WorkerGetRandomCandidateNode
- master_remove_node only complains about active shard placements if the
  node being removed is a primary.
- master_remove_node only deletes all reference table placements in the
  group if the node being removed is the primary.
- Rename {Node->NodeGroup}HasShardPlacements, this reflects the behavior it
  already had.
- Rename DeleteAllReferenceTablePlacementsFrom{Node->NodeGroup}. This also
  reflects the behavior it already had, but the new signature forces the
  caller to pass in a groupId
- Rename {WorkerGetLiveGroup->ActivePrimaryNode}Count
2017-07-24 11:57:46 +03:00
Brian Cloutier 7ad95b53d2 Rename pg_dist_shard_placement -> pg_dist_placement
Comes with a few changes:

- Change the signature of some functions to accept groupid
  - InsertShardPlacementRow
  - DeleteShardPlacementRow
  - UpdateShardPlacementState

- NodeHasActiveShardPlacements returns true if the group the node is a
  part of has any active shard placements

- TupleToShardPlacement now returns ShardPlacements which have NULL
  nodeName and nodePort.

- Populate (nodeName, nodePort) when creating ShardPlacements
- Disallow removing a node if it contains any shard placements

- DeleteAllReferenceTablePlacementsFromNode matches based on group. This
  doesn't change behavior for now (while there is only one node per
  group), but means in the future callers should be careful about
  calling it on a secondary node, it'll delete placements on the primary.

- Create concept of a GroupShardPlacement, which represents an actual
  tuple in pg_dist_placement and is distinct from a ShardPlacement,
  which has been resolved to a specific node. In the future
  ShardPlacement should be renamed to NodeShardPlacement.

- Create some triggers which allow existing code to continue to insert
  into and update pg_dist_shard_placement as if it still existed.
2017-07-12 14:17:31 +02:00
Murat Tuncer 2a4eada150 Replace duplicate code and call check_functions_in_node (#1478)
MasterIrreducibleExpressionWalker has a copied code from
function check_functions_in_node() which was available with
PG 9.6+. Now PG 9.5 support is dropped we can remove
duplicate code and directly call check_functions_in_node().
2017-07-07 10:19:33 +03:00
Marco Slot da47a03b18 Move INSERT ... SELECT planning logic into one place 2017-06-29 15:03:14 +02:00
Andres Freund b96ba9b490 Fix code only enabled for 9.5.
There's still supporting wrappers used, a subsequent commit will
remove those.

This also removes the already unused tuplecount_t define.
2017-06-26 08:46:32 -07:00
Jason Petersen 2204da19f0 Support PostgreSQL 10 (#1379)
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.
2017-06-26 02:35:46 -06:00
Marco Slot 2f8ac82660 Execute INSERT..SELECT via coordinator if it cannot be pushed down
Add a second implementation of INSERT INTO distributed_table SELECT ... that is used if
the query cannot be pushed down. The basic idea is to execute the SELECT query separately
and pass the results into the distributed table using a CopyDestReceiver, which is also
used for COPY and create_distributed_table. When planning the SELECT, we go through
planner hooks again, which means the SELECT can also be a distributed query.

EXPLAIN is supported, but EXPLAIN ANALYZE is not because preventing double execution was
a lot more complicated in this case.
2017-06-22 15:46:30 +02:00
Marco Slot 155db4d913 Simplify router planner call path 2017-06-22 15:45:57 +02:00
Önder Kalacı b74ed3c8e1 Subqueries in where -- updated (#1372)
* Support for subqueries in WHERE clause

This commit enables subqueries in WHERE clause to be pushed down
by the subquery pushdown logic.

The support covers:
  - Correlated subqueries with IN, NOT IN, EXISTS, NOT EXISTS,
    operator expressions such as (>, <, =, ALL, ANY etc.)
  - Non-correlated subqueries with (partition_key) IN (SELECT partition_key ..)
    (partition_key) =ANY (SELECT partition_key ...)

Note that this commit heavily utilizes the attribute equivalence logic introduced
in the 1cb6a34ba8. In general, this commit mostly
adjusts the logical planner not to error out on the subqueries in WHERE clause.

* Improve error checks for subquery pushdown and INSERT ... SELECT

Since we allow subqueries in WHERE clause with the previous commit,
we should apply the same limitations to those subqueries.

With this commit, we do not iterate on each subquery one by one.
Instead, we extract all the subqueries and apply the checks directly
on those subqueries. The aim of this change is to (i) Simplify the
code (ii) Make it close to the checks on INSERT .. SELECT code base.

* Extend checks for unresolved paramaters to include SubLinks

With the presence of subqueries in where clause (i.e., SubPlans on the
query) the existing way for checking unresolved parameters fail. The
reason is that the parameters for SubPlans are kept on the parent plan not
on the query itself (see primnodes.h for the details).

With this commit, instead of checking SubPlans on the modified plans
we start to use originalQuery, where SubLinks represent the subqueries
in where clause. The unresolved parameters can be found on the SubLinks.

* Apply code-review feedback

* Remove unnecessary copying of shard interval list

This commit removes unnecessary copying of shard interval list. Note
that there are no copyObject function implemented for shard intervals.
2017-05-01 17:20:21 +03:00
Önder Kalacı ad5cd326a4 Subquery pushdown - main branch (#1323)
* Enabling physical planner for subquery pushdown changes

This commit applies the logic that exists in INSERT .. SELECT
planning to the subquery pushdown changes.

The main algorithm is followed as :
   - pick an anchor relation (i.e., target relation)
   - per each target shard interval
       - add the target shard interval's shard range
         as a restriction to the relations (if all relations
         joined on the partition keys)
        - Check whether the query is router plannable per
          target shard interval.
        - If router plannable, create a task

* Add union support within the JOINS

This commit adds support for UNION/UNION ALL subqueries that are
in the following form:

     .... (Q1 UNION Q2 UNION ...) as union_query JOIN (QN) ...

In other words, we currently do NOT support the queries that are
in the following form where union query is not JOINed with
other relations/subqueries :

     .... (Q1 UNION Q2 UNION ...) as union_query ....

* Subquery pushdown planner uses original query

With this commit, we change the input to the logical planner for
subquery pushdown. Before this commit, the planner was relying
on the query tree that is transformed by the postgresql planner.
After this commit, the planner uses the original query. The main
motivation behind this change is the simplify deparsing of
subqueries.

* Enable top level subquery join queries

This work enables
- Top level subquery joins
- Joins between subqueries and relations
- Joins involving more than 2 range table entries

A new regression test file is added to reflect enabled test cases

* Add top level union support

This commit adds support for UNION/UNION ALL subqueries that are
in the following form:

     .... (Q1 UNION Q2 UNION ...) as union_query ....

In other words, Citus supports allow top level
unions being wrapped into aggregations queries
and/or simple projection queries that only selects
some fields from the lower level queries.

* Disallow subqueries without a relation in the range table list for subquery pushdown

This commit disallows subqueries without relation in the range table
list. This commit is only applied for subquery pushdown. In other words,
we do not add this limitation for single table re-partition subqueries.

The reasoning behind this limitation is that if we allow pushing down
such queries, the result would include (shardCount * expectedResults)
where in a non distributed world the result would be (expectedResult)
only.

* Disallow subqueries without a relation in the range table list for INSERT .. SELECT

This commit disallows subqueries without relation in the range table
list. This commit is only applied for INSERT.. SELECT queries.

The reasoning behind this limitation is that if we allow pushing down
such queries, the result would include (shardCount * expectedResults)
where in a non distributed world the result would be (expectedResult)
only.

* Change behaviour of subquery pushdown flag (#1315)

This commit changes the behaviour of the citus.subquery_pushdown flag.
Before this commit, the flag is used to enable subquery pushdown logic. But,
with this commit, that behaviour is enabled by default. In other words, the
flag is now useless. We prefer to keep the flag since we don't want to break
the backward compatibility. Also, we may consider using that flag for other
purposes in the next commits.

* Require subquery_pushdown when limit is used in subquery

Using limit in subqueries may cause returning incorrect
results. Therefore we allow limits in subqueries only
if user explicitly set subquery_pushdown flag.

* Evaluate expressions on the LIMIT clause (#1333)

Subquery pushdown uses orignal query, the LIMIT and OFFSET clauses
are not evaluated. However, logical optimizer expects these expressions
are already evaluated by the standard planner. This commit manually
evaluates the functions on the logical planner for subquery pushdown.

* Better format subquery regression tests (#1340)

* Style fix for subquery pushdown regression tests

With this commit we intented a more consistent style for the
regression tests we've added in the
  - multi_subquery_union.sql
  - multi_subquery_complex_queries.sql
  - multi_subquery_behavioral_analytics.sql

* Enable the tests that are temporarily commented

This commit enables some of the regression tests that were commented
out until all the development is done.

* Fix merge conflicts (#1347)

 - Update regression tests to meet the changes in the regression
   test output.
 - Replace Ifs with Asserts given that the check is already done
 - Update shard pruning outputs

* Add view regression tests for increased subquery coverage (#1348)

- joins between views and tables
- joins between views
- union/union all queries involving views
- views with limit
- explain queries with view

* Improve btree operators for the subquery tests

This commit adds the missing comprasion for subquery composite key
btree comparator.
2017-04-29 04:09:48 +03:00
Andres Freund d399f395f7 Faster shard pruning.
So far citus used postgres' predicate proofing logic for shard
pruning, except for INSERT and COPY which were already optimized for
speed.  That turns out to be too slow:
* Shard pruning for SELECTs is currently O(#shards), because
  PruneShardList calls predicate_refuted_by() for every
  shard. Obviously using an O(N) type algorithm for general pruning
  isn't good.
* predicate_refuted_by() is quite expensive on its own right. That's
  primarily because it's optimized for doing a single refutation
  proof, rather than performing the same proof over and over.
* predicate_refuted_by() does not keep persistent state (see 2.) for
  function calls, which means that a lot of syscache lookups will be
  performed. That's particularly bad if the partitioning key is a
  composite key, because without a persistent FunctionCallInfo
  record_cmp() has to repeatedly look-up the type definition of the
  composite key. That's quite expensive.

Thus replace this with custom-code that works in two phases:
1) Search restrictions for constraints that can be pruned upon
2) Use those restrictions to search for matching shards in the most
   efficient manner available:
   a) Binary search / Hash Lookup in case of hash partitioned tables
   b) Binary search for equal clauses in case of range or append
      tables without overlapping shards.
   c) Binary search for inequality clauses, searching for both lower
      and upper boundaries, again in case of range or append
      tables without overlapping shards.
   d) exhaustive search testing each ShardInterval

My measurements suggest that we are considerably, often orders of
magnitude, faster than the previous solution, even if we have to fall
back to exhaustive pruning.
2017-04-28 14:40:41 -07:00
Jason Petersen 93e3afc25c
Remove FastShardPruning method
With the other simplifications, it doesn't make sense to keep around.
2017-04-27 13:32:36 -06:00
Jason Petersen 42ee7c05f5
Refactor FindShardInterval to use cacheEntry
All callers fetch a cache entry and extract/compute arguments for the
eventual FindShardInterval call, so it makes more sense to refactor
into that function itself; this solves the use-after-free bug, too.
2017-04-27 13:32:36 -06:00
Marco Slot 4ed093970a Support expressions in the partition column in INSERTs 2017-04-21 14:05:52 +02:00
velioglu 8cbef819be Log message of across shard queries according to the log level 2017-04-20 12:24:46 +03:00
velioglu 2327b63291 Change native hash function with worker_hash 2017-04-19 22:16:55 +03:00
Marco Slot af0e462409 Support UPDATE/DELETE with parameterised partition column qual 2017-04-17 16:17:30 +02:00
Burak Yucesoy e9095e62ec Decouple reference table replication
With this change we add an option to add a node without replicating all reference
tables to that node. If a node is added with this option, we mark the node as
inactive and no queries will sent to that node.

We also added two new UDFs;
 - master_activate_node(host, port):
    - marks node as active and replicates all reference tables to that node
 - master_add_inactive_node(host, port):
    - only adds node to pg_dist_node
2017-04-17 13:33:31 +03:00
Onder Kalaci 1cb6a34ba8 Remove uninstantiated qual logic, use attribute equivalences
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.
2017-04-13 11:51:26 +03:00
Onder Kalaci 11665dbe3c Fix pushing down wrong queries for INSERT ... SELECT queries
Before this commit, in certain cases router planner allowed pushing
down JOINs that are not on the partition keys.

With @anarazel's suggestion, we change the logic to use uninstantiated
parameter. Previously, the planner was traversing on the restriction
information and once it finds the parameter, it was replacing it with
the shard range. With this commit, instead of traversing the restrict
infos, the planner explicitly checks for the equivalence of the relation
partition key with the uninstantiated parameter. If finds an equivalence,
it adds the restrictions. In this way, we have more control over the
queries that are pushed down.
2017-03-24 11:37:35 +02:00
Murat Tuncer c4734d7d94 Rephrase router modify errors
generic "distributed modifications must target exactly one shard"
message is replaced by more context aware error messages.
2017-03-16 15:09:10 +03:00
Metin Doslu 1f838199f8 Use CustomScan API for query execution
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.
2017-03-14 12:17:51 +02:00
Andres Freund 52358fe891 Initial temp table removal implementation 2017-03-14 12:09:49 +02:00
Murat Tuncer f657a744d5 Enable router planner for queries on range partitioned tables
Router planner now supports queries using range partitioned
tables. Queries on append partitioned tables are still not
supported.
2017-03-09 16:39:15 +03:00