Add an infrastructure to run same tests with arbitrary configs (#5316)

To run tests in parallel use:

```bash
make check-arbitrary-configs parallel=4
```

To run tests sequentially use:

```bash
make check-arbitrary-configs parallel=1
```

To run only some configs:

```bash
make check-arbitrary-base CONFIGS=CitusSingleNodeClusterConfig,CitusSmallSharedPoolSizeConfig
```

To run only some test files with some config:

```bash
make check-arbitrary-base CONFIGS=CitusSingleNodeClusterConfig EXTRA_TESTS=dropped_columns_1
```

To get a deterministic run, you can give the random's seed:

```bash
make check-arbitrary-configs parallel=4 seed=12312
```

The `seed` will be in the output of the run.

In our regular regression tests, we can see all the details about either planning or execution but this means
we need to run the same query under different configs/cluster setups again and again, which is not really maintanable.

When we don't care about the internals of how planning/execution is done but the correctness, especially with different configs
this infrastructure can be used.

With `check-arbitrary-configs` target, the following happens:

-   a bunch of configs are loaded, which are defined in `config.py`. These configs have different settings such as different shard count, different citus settings, postgres settings, worker amount, or different metadata.
-   For each config, a separate data directory is created for tests in `tmp_citus_test` with the config's name.
-   For each config, `create_schedule` is run on the coordinator to setup the necessary tables.
-   For each config, `sql_schedule` is run. `sql_schedule` is run on the coordinator if it is a non-mx cluster. And if it is mx, it is either run on the coordinator or a random worker.
-   Tests results are checked if they match with the expected.

When tests results don't match, you can see the regression diffs in a config's datadir, such as `tmp_citus_tests/dataCitusSingleNodeClusterConfig`.

We also have a PostgresConfig which runs all the test suite with Postgres.
By default configs use regular user, but we have a config to run as a superuser as well.

So the infrastructure tests:

-   Postgres vs Citus
-   Mx vs Non-Mx
-   Superuser vs regular user
-   Arbitrary Citus configs

When you want to add a new test, you can add the create statements to `create_schedule` and add the sql queries to `sql_schedule`.
If you are adding Citus UDFs that should be a NO-OP for Postgres, make sure to override the UDFs in `postgres.sql`.

You can add your new config to `config.py`. Make sure to extend either `CitusDefaultClusterConfig` or `CitusMXBaseClusterConfig`.

On the CI, upon a failure, all logfiles will be uploaded as artifacts, so you can check the artifacts tab.
All the regressions will be shown as part of the job on CI.

In your local, you can check the regression diffs in config's datadirs as in `tmp_citus_tests/dataCitusSingleNodeClusterConfig`.
pull/5367/head
SaitTalhaNisanci 2021-10-12 14:24:19 +03:00 committed by GitHub
parent a8348047c5
commit 3f65751d43
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
68 changed files with 7651 additions and 432 deletions

View File

@ -169,6 +169,82 @@ jobs:
- codecov/upload:
flags: 'test_<< parameters.old_pg_major >>_<< parameters.new_pg_major >>,upgrade'
test-arbitrary-configs:
description: Runs tests on arbitrary configs
parameters:
pg_major:
description: 'postgres major version to use'
type: integer
image:
description: 'docker image to use as for the tests'
type: string
default: citus/failtester
image_tag:
description: 'docker image tag to use'
type: string
default: 12-13
docker:
- image: '<< parameters.image >>:<< parameters.image_tag >>'
resource_class: xlarge
working_directory: /home/circleci/project
steps:
- checkout
- attach_workspace:
at: .
- run:
name: 'Install Extension'
command: |
tar xfv "${CIRCLE_WORKING_DIRECTORY}/install-<< parameters.pg_major >>.tar" --directory /
- run:
name: 'Configure'
command: |
chown -R circleci .
gosu circleci ./configure
- run:
name: 'Enable core dumps'
command: |
ulimit -c unlimited
- run:
name: 'Test arbitrary configs'
command: |
gosu circleci \
make -C src/test/regress \
check-arbitrary-configs parallel=4
no_output_timeout: 2m
- run:
name: 'Show regressions'
command: |
find src/test/regress/tmp_citus_test/ -name "regression*.diffs" -exec cat {} +
lines=$(find src/test/regress/tmp_citus_test/ -name "regression*.diffs" | wc -l)
if [ $lines -ne 0 ]; then
exit 1
fi
when: on_fail
- run:
name: 'Copy logfiles'
command: |
mkdir src/test/regress/tmp_citus_test/logfiles
find src/test/regress/tmp_citus_test/ -name "logfile_*" -exec cp -t src/test/regress/tmp_citus_test/logfiles/ {} +
when: on_fail
- run:
name: 'Copy coredumps'
command: |
mkdir -p /tmp/core_dumps
if ls core.* 1> /dev/null 2>&1; then
cp core.* /tmp/core_dumps
fi
when: on_fail
- store_artifacts:
name: 'Save core dumps'
path: /tmp/core_dumps
- store_artifacts:
name: "Save logfiles"
path: src/test/regress/tmp_citus_test/logfiles
- codecov/upload:
flags: 'test_<< parameters.pg_major >>,upgrade'
test-citus-upgrade:
description: Runs citus upgrade tests
parameters:
@ -678,6 +754,22 @@ workflows:
make: check-failure
requires: [build-14]
- test-arbitrary-configs:
name: 'test-12_check-arbitrary-configs'
pg_major: 12
image_tag: '12.8'
requires: [build-12]
- test-arbitrary-configs:
name: 'test-13_check-arbitrary-configs'
pg_major: 13
image_tag: '13.4'
requires: [build-13]
- test-arbitrary-configs:
name: 'test-14_check-arbitrary-configs'
pg_major: 14
image_tag: '14.0'
requires: [build-14]
- test-pg-upgrade:
name: 'test-12-13_check-pg-upgrade'
old_pg_major: 12

View File

@ -6,6 +6,7 @@
/tmp_upgrade/
/tmp_citus_upgrade/
/tmp_citus_tarballs/
/tmp_citus_test/
/build/
/results/
/log/

View File

@ -27,8 +27,9 @@ MULTI_INSTALLDIR=$(CURDIR)/tmp_check/install
pg_regress_multi_check = $(PERL) $(citus_abs_srcdir)/pg_regress_multi.pl --pgxsdir="$(pgxsdir)" --bindir="$(bindir)" --libdir="$(libdir)" --majorversion="$(MAJORVERSION)" --postgres-builddir="$(postgres_abs_builddir)" --postgres-srcdir="$(postgres_abs_srcdir)"
MULTI_REGRESS_OPTS = --inputdir=$(citus_abs_srcdir) $(pg_regress_locale_flags) --launcher="$(citus_abs_srcdir)/log_test_times"
pg_upgrade_check = $(citus_abs_srcdir)/upgrade/pg_upgrade_test.py
citus_upgrade_check = $(citus_abs_srcdir)/upgrade/citus_upgrade_test.py
pg_upgrade_check = $(citus_abs_srcdir)/citus_tests/upgrade/pg_upgrade_test.py
citus_upgrade_check = $(citus_abs_srcdir)/citus_tests/upgrade/citus_upgrade_test.py
arbitrary_config_check = $(citus_abs_srcdir)/citus_tests/arbitrary_configs/citus_arbitrary_configs.py
template_isolation_files = $(shell find $(citus_abs_srcdir)/spec/ -name '*.spec')
generated_isolation_files = $(patsubst $(citus_abs_srcdir)/spec/%,$(citus_abs_srcdir)/build/specs/%,$(template_isolation_files))
@ -209,6 +210,12 @@ check-failure-base: all
check-pg-upgrade:
$(pg_upgrade_check) --old-bindir=$(old-bindir) --new-bindir=$(new-bindir) --pgxsdir=$(pgxsdir)
check-arbitrary-configs:
${arbitrary_config_check} --bindir=$(bindir) --pgxsdir=$(pgxsdir) --parallel=$(parallel) --configs=$(CONFIGS) --seed=$(seed)
check-arbitrary-base:
${arbitrary_config_check} --bindir=$(bindir) --pgxsdir=$(pgxsdir) --parallel=$(parallel) --configs=$(CONFIGS) --seed=$(seed) --base
check-citus-upgrade:
$(citus_upgrade_check) \
--bindir=$(bindir) \
@ -243,5 +250,6 @@ clean-upgrade-artifacts:
clean distclean maintainer-clean:
rm -f $(output_files) $(input_files)
rm -rf tmp_check/
rm -rf tmp_citus_test/
all: create-tablespaces

View File

@ -0,0 +1,13 @@
#!/bin/bash
set -euo pipefail
## Set mydir to the directory containing the script
## The ${var%pattern} format will remove the shortest match of
## pattern from the end of the string. Here, it will remove the
## script's name,. leaving only the directory.
datadir="${0%/*}"
cd "${datadir}"
# shellcheck source=copy_modified
source copy_modified

View File

View File

@ -0,0 +1,81 @@
# Arbitrary Configs
## Usage
To run tests in parallel use:
```bash
# will run 4 configs in parallel
make check-arbitrary-configs parallel=4
```
To run tests sequentially use:
```bash
make check-arbitrary-configs parallel=1
```
To run only some configs:
```bash
# Config names should be comma separated
make check-arbitrary-base CONFIGS=CitusSingleNodeClusterConfig,CitusSmallSharedPoolSizeConfig
```
To run only some test files with some config:
```bash
make check-arbitrary-base CONFIGS=CitusSingleNodeClusterConfig EXTRA_TESTS=dropped_columns_1
```
To get a deterministic run, you can give the random's seed:
```bash
make check-arbitrary-configs parallel=4 seed=12312
```
The `seed` will be in the output of the run.
## General Info
In our regular regression tests, we can see all the details about either planning or execution but this means
we need to run the same query under different configs/cluster setups again and again, which is not really maintanable.
When we don't care about the internals of how planning/execution is done but the correctness, especially with different configs
this infrastructure can be used.
With `check-arbitrary-configs` target, the following happens:
- a bunch of configs are loaded, which are defined in `config.py`. These configs have different settings such as different shard count, different citus settings, postgres settings, worker amount, or different metadata.
- For each config, a separate data directory is created for tests in `tmp_citus_test` with the config's name.
- For each config, `create_schedule` is run on the coordinator to setup the necessary tables.
- For each config, `sql_schedule` is run. `sql_schedule` is run on the coordinator if it is a non-mx cluster. And if it is mx, it is either run on the coordinator or a random worker.
- Tests results are checked if they match with the expected.
When tests results don't match, you can see the regression diffs in a config's datadir, such as `tmp_citus_tests/dataCitusSingleNodeClusterConfig`.
We also have a PostgresConfig which runs all the test suite with Postgres.
By default configs use regular user, but we have a config to run as a superuser as well.
So the infrastructure tests:
- Postgres vs Citus
- Mx vs Non-Mx
- Superuser vs regular user
- Arbitrary Citus configs
## Adding a new test
When you want to add a new test, you can add the create statements to `create_schedule` and add the sql queries to `sql_schedule`.
If you are adding Citus UDFs that should be a NO-OP for Postgres, make sure to override the UDFs in `postgres.sql`.
## Adding a new config
You can add your new config to `config.py`. Make sure to extend either `CitusDefaultClusterConfig` or `CitusMXBaseClusterConfig`.
## Debugging failures
On the CI, upon a failure, all logfiles will be uploaded as artifacts, so you can check the artifacts tab.
All the regressions will be shown as part of the job on CI.
In your local, you can check the regression diffs in config's datadirs as in `tmp_citus_tests/dataCitusSingleNodeClusterConfig`.

View File

@ -0,0 +1,239 @@
#!/usr/bin/env python3
"""citus_arbitrary_configs
Usage:
citus_arbitrary_configs --bindir=<bindir> --pgxsdir=<pgxsdir> --parallel=<parallel> --configs=<configs> --seed=<seed> [--base]
Options:
--bindir=<bindir> The PostgreSQL executable directory(ex: '~/.pgenv/pgsql-11.3/bin')
--pgxsdir=<pgxsdir> Path to the PGXS directory(ex: ~/.pgenv/src/postgresql-11.3)
--parallel=<parallel> how many configs to run in parallel
--configs=<configs> the config names to run
--seed=<seed> random number seed
--base whether to use the base sql schedule or not
"""
import sys
import os, shutil
# https://stackoverflow.com/questions/14132789/relative-imports-for-the-billionth-time/14132912#14132912
sys.path.append(os.path.dirname(os.path.dirname(os.path.abspath(__file__))))
import common
import config as cfg
import concurrent.futures
import multiprocessing
from docopt import docopt
import time
import inspect
import random
testResults = {}
parallel_thread_amount = 1
def _run_pg_regress_on_port(config, port, schedule_name, extra_tests=""):
return common.run_pg_regress_without_exit(
config.bindir,
config.pg_srcdir,
port,
schedule_name,
config.output_dir,
config.input_dir,
config.user,
extra_tests,
)
def run_for_config(config, lock, sql_schedule_name):
name = config.name
print("Running test for: {}".format(name))
start_time = time.time()
common.initialize_citus_cluster(
config.bindir, config.datadir, config.settings, config
)
if config.user == cfg.REGULAR_USER_NAME:
common.create_role(
config.bindir,
config.coordinator_port(),
config.node_name_to_ports.values(),
config.user,
)
copy_copy_modified_binary(config.datadir)
copy_test_files(config)
exitCode = 0
if not config.is_citus:
exitCode |= common.run_pg_regress_without_exit(
config.bindir,
config.pg_srcdir,
config.coordinator_port(),
cfg.POSTGRES_SCHEDULE,
config.output_dir,
config.input_dir,
cfg.SUPER_USER_NAME,
)
common.save_regression_diff("postgres", config.output_dir)
exitCode |= _run_pg_regress_on_port(
config, config.coordinator_port(), cfg.CREATE_SCHEDULE
)
common.save_regression_diff("create", config.output_dir)
extra_tests = os.getenv("EXTRA_TESTS", "")
if config.is_mx and config.worker_amount > 0:
exitCode |= _run_pg_regress_on_port(
config, config.random_port(), sql_schedule_name, extra_tests=extra_tests
)
else:
exitCode |= _run_pg_regress_on_port(
config,
config.coordinator_port(),
sql_schedule_name,
extra_tests=extra_tests,
)
run_time = time.time() - start_time
with lock:
testResults[name] = (
"SUCCESS"
if exitCode == 0
else "FAIL: see {}".format(config.output_dir + "/run.out")
)
testResults[name] += " runtime: {} seconds".format(run_time)
common.stop_databases(
config.bindir, config.datadir, config.node_name_to_ports, config.name
)
common.save_regression_diff("sql", config.output_dir)
return exitCode
def copy_copy_modified_binary(datadir):
shutil.copy("bin/copy_modified", datadir)
shutil.copy("bin/copy_modified_wrapper", datadir)
def copy_test_files(config):
sql_dir_path = os.path.join(config.datadir, "sql")
expected_dir_path = os.path.join(config.datadir, "expected")
common.initialize_temp_dir(sql_dir_path)
common.initialize_temp_dir(expected_dir_path)
for scheduleName in cfg.ARBITRARY_SCHEDULE_NAMES:
with open(scheduleName) as file:
lines = file.readlines()
for line in lines:
colon_index = line.index(":")
line = line[colon_index + 1 :].strip()
test_names = line.split(" ")
copy_test_files_with_names(test_names, sql_dir_path, expected_dir_path)
def copy_test_files_with_names(test_names, sql_dir_path, expected_dir_path):
for test_name in test_names:
sql_name = os.path.join("./sql", test_name + ".sql")
output_name = os.path.join("./expected", test_name + ".out")
shutil.copy(sql_name, sql_dir_path)
if os.path.isfile(output_name):
# it might be the first time we run this test and the expected file
# might not be there yet, in that case, we don't want to error out
# while copying the file.
shutil.copy(output_name, expected_dir_path)
def run_tests(configs, sql_schedule_name):
failCount = 0
common.initialize_temp_dir(cfg.CITUS_ARBITRARY_TEST_DIR)
with concurrent.futures.ThreadPoolExecutor(
max_workers=parallel_thread_amount
) as executor:
manager = multiprocessing.Manager()
lock = manager.Lock()
futures = [
executor.submit(run_for_config, config, lock, sql_schedule_name)
for config in configs
]
try:
for future in futures:
exitCode = future.result()
if exitCode != 0:
failCount += 1
except KeyboardInterrupt:
exit(1)
return failCount
def read_configs(docoptRes):
configs = []
# We fill the configs from all of the possible classes in config.py so that if we add a new config,
# we don't need to add it here. And this avoids the problem where we forget to add it here
for x in cfg.__dict__.values():
if inspect.isclass(x) and (
issubclass(x, cfg.CitusMXBaseClusterConfig)
or issubclass(x, cfg.CitusDefaultClusterConfig)
):
configs.append(x(docoptRes))
return configs
def read_arguments(docoptRes):
global parallel_thread_amount
if "--parallel" in docoptRes and docoptRes["--parallel"] != "":
parallel_thread_amount = int(docoptRes["--parallel"])
seed = random.randint(1, 1000000)
if "--seed" in docoptRes and docoptRes["--seed"] != "":
seed = int(docoptRes["--seed"])
random.seed(seed)
configs = read_configs(docoptRes)
if "--configs" in docoptRes and docoptRes["--configs"] != "":
given_configs = docoptRes["--configs"].split(",")
new_configs = []
for config in configs:
if config.name in given_configs:
new_configs.append(config)
if len(new_configs) > 0:
configs = new_configs
sql_schedule_name = cfg.SQL_SCHEDULE
if "--base" in docoptRes:
sql_schedule_name = cfg.SQL_BASE_SCHEDULE
return configs, sql_schedule_name, seed
def show_results(configs, testResults, runtime, seed):
for testName, testResult in testResults.items():
print("{}: {}".format(testName, testResult))
print("--- {} seconds to run all tests! ---".format(end_time - start_time))
print("---SEED: {} ---".format(seed))
configCount = len(configs)
if len(testResults) != configCount or failCount > 0:
print(
"actual {} expected {}, failCount: {}".format(
len(testResults), configCount, failCount
)
)
sys.exit(1)
if __name__ == "__main__":
docoptRes = docopt(__doc__)
start_time = time.time()
configs, sql_schedule_name, seed = read_arguments(docoptRes)
failCount = run_tests(configs, sql_schedule_name)
end_time = time.time()
show_results(configs, testResults, end_time - start_time, seed)

View File

@ -0,0 +1,251 @@
import os
import shutil
import sys
import subprocess
import atexit
import utils
from utils import USER, cd
def initialize_temp_dir(temp_dir):
if os.path.exists(temp_dir):
shutil.rmtree(temp_dir)
os.mkdir(temp_dir)
# Give full access to TEMP_DIR so that postgres user can use it.
os.chmod(temp_dir, 0o777)
def initialize_temp_dir_if_not_exists(temp_dir):
if os.path.exists(temp_dir):
return
os.mkdir(temp_dir)
# Give full access to TEMP_DIR so that postgres user can use it.
os.chmod(temp_dir, 0o777)
def initialize_db_for_cluster(pg_path, rel_data_path, settings, node_names):
subprocess.run(["mkdir", rel_data_path], check=True)
for node_name in node_names:
abs_data_path = os.path.abspath(os.path.join(rel_data_path, node_name))
command = [
os.path.join(pg_path, "initdb"),
"--pgdata",
abs_data_path,
"--username",
USER,
]
subprocess.run(command, check=True)
add_settings(abs_data_path, settings)
def add_settings(abs_data_path, settings):
conf_path = os.path.join(abs_data_path, "postgresql.conf")
with open(conf_path, "a") as conf_file:
for setting_key, setting_val in settings.items():
setting = "{setting_key} = '{setting_val}'\n".format(
setting_key=setting_key, setting_val=setting_val
)
conf_file.write(setting)
def create_role(pg_path, port, node_ports, user_name):
for port in node_ports:
command = "SELECT worker_create_or_alter_role('{}', 'CREATE ROLE {} WITH LOGIN CREATEROLE CREATEDB;', NULL)".format(
user_name, user_name
)
utils.psql(pg_path, port, command)
command = "GRANT CREATE ON DATABASE postgres to {}".format(user_name)
utils.psql(pg_path, port, command)
def coordinator_should_haveshards(pg_path, port):
command = "SELECT citus_set_node_property('localhost', {}, 'shouldhaveshards', true)".format(
port
)
utils.psql(pg_path, port, command)
def start_databases(pg_path, rel_data_path, node_name_to_ports, logfile_prefix):
for node_name in node_name_to_ports.keys():
abs_data_path = os.path.abspath(os.path.join(rel_data_path, node_name))
node_port = node_name_to_ports[node_name]
command = [
os.path.join(pg_path, "pg_ctl"),
"start",
"--pgdata",
abs_data_path,
"-U",
USER,
"-o",
"-p {}".format(node_port),
"--log",
os.path.join(abs_data_path, logfile_name(logfile_prefix, node_name)),
]
subprocess.run(command, check=True)
atexit.register(
stop_databases,
pg_path,
rel_data_path,
node_name_to_ports,
logfile_prefix,
no_output=True,
)
def create_citus_extension(pg_path, node_ports):
for port in node_ports:
utils.psql(pg_path, port, "CREATE EXTENSION citus;")
def run_pg_regress(pg_path, pg_srcdir, port, schedule):
should_exit = True
_run_pg_regress(pg_path, pg_srcdir, port, schedule, should_exit)
subprocess.run("bin/copy_modified", check=True)
def run_pg_regress_without_exit(
pg_path,
pg_srcdir,
port,
schedule,
output_dir=".",
input_dir=".",
user="postgres",
extra_tests="",
):
should_exit = False
exit_code = _run_pg_regress(
pg_path,
pg_srcdir,
port,
schedule,
should_exit,
output_dir,
input_dir,
user,
extra_tests,
)
copy_binary_path = os.path.join(input_dir, "copy_modified_wrapper")
exit_code |= subprocess.call(copy_binary_path)
return exit_code
def _run_pg_regress(
pg_path,
pg_srcdir,
port,
schedule,
should_exit,
output_dir=".",
input_dir=".",
user="postgres",
extra_tests="",
):
command = [
os.path.join(pg_srcdir, "src/test/regress/pg_regress"),
"--port",
str(port),
"--schedule",
schedule,
"--bindir",
pg_path,
"--user",
user,
"--dbname",
"postgres",
"--inputdir",
input_dir,
"--outputdir",
output_dir,
"--use-existing",
]
if extra_tests != "":
command.append(extra_tests)
exit_code = subprocess.call(command)
if should_exit and exit_code != 0:
sys.exit(exit_code)
return exit_code
def save_regression_diff(name, output_dir):
path = os.path.join(output_dir, "regression.diffs")
if not os.path.exists(path):
return
new_file_path = os.path.join(output_dir, "./regression_{}.diffs".format(name))
print("new file path:", new_file_path)
shutil.move(path, new_file_path)
def sync_metadata_to_workers(pg_path, worker_ports, coordinator_port):
for port in worker_ports:
command = (
"SELECT * from start_metadata_sync_to_node('localhost', {port});".format(
port=port
)
)
utils.psql(pg_path, coordinator_port, command)
def add_coordinator_to_metadata(pg_path, coordinator_port):
command = "SELECT citus_add_node('localhost', {}, groupId := 0)".format(
coordinator_port
)
utils.psql(pg_path, coordinator_port, command)
def add_workers(pg_path, worker_ports, coordinator_port):
for port in worker_ports:
command = "SELECT * from master_add_node('localhost', {port});".format(
port=port
)
utils.psql(pg_path, coordinator_port, command)
def logfile_name(logfile_prefix, node_name):
return "logfile_" + logfile_prefix + "_" + node_name
def stop_databases(
pg_path, rel_data_path, node_name_to_ports, logfile_prefix, no_output=False
):
for node_name in node_name_to_ports.keys():
abs_data_path = os.path.abspath(os.path.join(rel_data_path, node_name))
node_port = node_name_to_ports[node_name]
command = [
os.path.join(pg_path, "pg_ctl"),
"stop",
"--pgdata",
abs_data_path,
"-U",
USER,
"-o",
"-p {}".format(node_port),
"--log",
os.path.join(abs_data_path, logfile_name(logfile_prefix, node_name)),
]
if no_output:
subprocess.call(
command, stdout=subprocess.DEVNULL, stderr=subprocess.DEVNULL
)
else:
subprocess.call(command)
def initialize_citus_cluster(bindir, datadir, settings, config):
# In case there was a leftover from previous runs, stop the databases
stop_databases(
bindir, datadir, config.node_name_to_ports, config.name, no_output=True
)
initialize_db_for_cluster(
bindir, datadir, settings, config.node_name_to_ports.keys()
)
start_databases(bindir, datadir, config.node_name_to_ports, config.name)
create_citus_extension(bindir, config.node_name_to_ports.values())
add_workers(bindir, config.worker_ports, config.coordinator_port())
if config.is_mx:
sync_metadata_to_workers(bindir, config.worker_ports, config.coordinator_port())
if config.add_coordinator_to_metadata:
add_coordinator_to_metadata(bindir, config.coordinator_port())
config.setup_steps()

View File

@ -0,0 +1,332 @@
from os.path import expanduser
import random
import socket
from contextlib import closing
import os
import threading
import common
COORDINATOR_NAME = "coordinator"
WORKER1 = "worker1"
WORKER2 = "worker2"
REGULAR_USER_NAME = "regularuser"
SUPER_USER_NAME = "postgres"
ARBITRARY_SCHEDULE_NAMES = [
"create_schedule",
"sql_schedule",
"sql_base_schedule",
"postgres_schedule",
]
BEFORE_PG_UPGRADE_SCHEDULE = "./before_pg_upgrade_schedule"
AFTER_PG_UPGRADE_SCHEDULE = "./after_pg_upgrade_schedule"
CREATE_SCHEDULE = "./create_schedule"
POSTGRES_SCHEDULE = "./postgres_schedule"
SQL_SCHEDULE = "./sql_schedule"
SQL_BASE_SCHEDULE = "./sql_base_schedule"
AFTER_CITUS_UPGRADE_COORD_SCHEDULE = "./after_citus_upgrade_coord_schedule"
BEFORE_CITUS_UPGRADE_COORD_SCHEDULE = "./before_citus_upgrade_coord_schedule"
MIXED_BEFORE_CITUS_UPGRADE_SCHEDULE = "./mixed_before_citus_upgrade_schedule"
MIXED_AFTER_CITUS_UPGRADE_SCHEDULE = "./mixed_after_citus_upgrade_schedule"
CITUS_ARBITRARY_TEST_DIR = "./tmp_citus_test"
MASTER = "master"
# This should be updated when citus version changes
MASTER_VERSION = "11.0"
HOME = expanduser("~")
CITUS_VERSION_SQL = "SELECT extversion FROM pg_extension WHERE extname = 'citus';"
# this is out of ephemeral port range for many systems hence
# it is a lower change that it will conflict with "in-use" ports
next_port = 10200
# ephemeral port start on many linux systems
PORT_UPPER = 32768
port_lock = threading.Lock()
def find_free_port():
global next_port
with port_lock:
while next_port < PORT_UPPER:
with closing(socket.socket(socket.AF_INET, socket.SOCK_STREAM)) as s:
try:
s.bind(("localhost", next_port))
port = next_port
next_port += 1
return port
except:
next_port += 1
# we couldn't find a port
raise Exception("Couldn't find a port to use")
class NewInitCaller(type):
def __call__(cls, *args, **kwargs):
obj = type.__call__(cls, *args, **kwargs)
obj.post_init()
return obj
class CitusBaseClusterConfig(object, metaclass=NewInitCaller):
def __init__(self, arguments):
if "--bindir" in arguments:
self.bindir = arguments["--bindir"]
self.pg_srcdir = arguments["--pgxsdir"]
self.temp_dir = CITUS_ARBITRARY_TEST_DIR
self.worker_amount = 2
self.user = REGULAR_USER_NAME
self.is_mx = False
self.is_citus = True
self.name = type(self).__name__
self.settings = {
"shared_preload_libraries": "citus",
"log_error_verbosity": "terse",
"citus.node_conninfo": "sslmode=prefer",
"citus.enable_repartition_joins": True,
"citus.repartition_join_bucket_count_per_node": 2,
"max_connections": 600,
}
self.new_settings = {}
self.add_coordinator_to_metadata = False
def post_init(self):
self._init_node_name_ports()
self.datadir = self.temp_dir + "/data"
self.datadir += self.name
self.input_dir = self.datadir
self.output_dir = self.datadir
self.output_file = os.path.join(self.datadir, "run.out")
if self.worker_amount > 0:
self.chosen_random_worker_port = self.random_worker_port()
self.settings.update(self.new_settings)
def coordinator_port(self):
return self.node_name_to_ports[COORDINATOR_NAME]
def setup_steps(self):
pass
def random_worker_port(self):
return random.choice(self.worker_ports)
def random_port(self):
return random.choice(list(self.node_name_to_ports.values()))
def _init_node_name_ports(self):
self.node_name_to_ports = {}
self.worker_ports = []
cur_port = self._get_and_update_next_port()
self.node_name_to_ports[COORDINATOR_NAME] = cur_port
for i in range(self.worker_amount):
cur_port = self._get_and_update_next_port()
cur_worker_name = "worker{}".format(i)
self.node_name_to_ports[cur_worker_name] = cur_port
self.worker_ports.append(cur_port)
def _get_and_update_next_port(self):
if hasattr(self, "fixed_port"):
next_port = self.fixed_port
self.fixed_port += 1
return next_port
return find_free_port()
class CitusDefaultClusterConfig(CitusBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
new_settings = {
"client_min_messages": "WARNING",
"citus.sort_returning": True,
}
self.settings.update(new_settings)
self.add_coordinator_to_metadata = True
class CitusMXBaseClusterConfig(CitusDefaultClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.is_mx = True
class CitusUpgradeConfig(CitusBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.pre_tar_path = arguments["--citus-pre-tar"]
self.post_tar_path = arguments["--citus-post-tar"]
self.temp_dir = "./tmp_citus_upgrade"
self.new_settings = {"citus.enable_version_checks": "false"}
self.user = SUPER_USER_NAME
self.mixed_mode = arguments["--mixed"]
self.fixed_port = 57635
class PostgresConfig(CitusDefaultClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.worker_amount = 0
self.is_citus = False
class CitusSingleNodeClusterConfig(CitusDefaultClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.worker_amount = 0
def setup_steps(self):
common.coordinator_should_haveshards(self.bindir, self.coordinator_port())
class CitusSingleWorkerClusterConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.worker_amount = 1
class CitusSuperUserDefaultClusterConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.user = SUPER_USER_NAME
class CitusThreeWorkersManyShardsClusterConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {"citus.shard_count": 191}
self.worker_amount = 3
def setup_steps(self):
common.coordinator_should_haveshards(self.bindir, self.coordinator_port())
class CitusSmallSharedPoolSizeConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {
# can be uncommented after https://github.com/citusdata/citus/issues/5342
# "citus.local_shared_pool_size": 5,
"citus.max_shared_pool_size": 5,
}
class CitusSmallExecutorPoolSizeConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {
"citus.max_adaptive_executor_pool_size": 2,
}
class CitusSequentialExecutionConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {
"citus.multi_shard_modify_mode": "sequential",
}
class CitusCacheManyConnectionsConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {
"citus.max_cached_conns_per_worker": 4,
}
class CitusUnusualExecutorConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {
"citus.max_adaptive_executor_pool_size": 7,
"citus.executor_slow_start_interval": 1,
"citus.prevent_incomplete_connection_establishment": False,
"citus.enable_cost_based_connection_establishment": False,
"citus.max_cached_connection_lifetime": "10ms",
# https://github.com/citusdata/citus/issues/5345
# "citus.force_max_query_parallelization": "on",
"citus.binary_worker_copy_format": False,
"citus.enable_binary_protocol": False,
}
class CitusCacheManyConnectionsConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {
"citus.copy_switchover_threshold": "1B",
"citus.local_copy_flush_threshold": "1B",
"citus.remote_copy_flush_threshold": "1B",
}
class CitusSmallCopyBuffersConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {
"citus.copy_switchover_threshold": "1B",
"citus.local_copy_flush_threshold": "1B",
"citus.remote_copy_flush_threshold": "1B",
}
class CitusUnusualQuerySettingsConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {
"citus.task_assignment_policy": "first-replica",
"citus.enable_fast_path_router_planner": False,
"citus.enable_local_execution": False,
"citus.enable_single_hash_repartition_joins": True,
"citus.recover_2pc_interval": "1s",
"citus.remote_task_check_interval": "1ms",
}
class CitusSingleNodeSingleShardClusterConfig(CitusDefaultClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.worker_amount = 0
self.new_settings = {"citus.shard_count": 1}
def setup_steps(self):
common.coordinator_should_haveshards(self.bindir, self.coordinator_port())
class CitusShardReplicationFactorClusterConfig(CitusDefaultClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {"citus.shard_replication_factor": 2}
class CitusSingleShardClusterConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.new_settings = {"citus.shard_count": 1}
class CitusNonMxClusterConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.is_mx = False
class PGUpgradeConfig(CitusBaseClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.old_bindir = arguments["--old-bindir"]
self.new_bindir = arguments["--new-bindir"]
self.temp_dir = "./tmp_upgrade"
self.old_datadir = self.temp_dir + "/oldData"
self.new_datadir = self.temp_dir + "/newData"
self.user = SUPER_USER_NAME

View File

@ -0,0 +1,110 @@
# Upgrade Tests
## Postgres Upgrade Test
Postgres upgrade test is used for testing postgres version upgrade with citus installed.
Before running the script, make sure that:
- You have downloaded citus.
- You have two different postgres versions.
- Citus is installed to both of the postgres versions. For each postgres version:
- In citus source directory run:
```bash
make clean
./configure PG_CONFIG=<your path to postgres pg config>
PG_CONFIG=<your path to postgres pg config> make
sudo PG_CONFIG=<your path to postgres pg config> make install
```
Make sure you do this for both postgres versions, pg_config should be different for each postgres version.
- Install `pipenv` and run in `citus/src/test/regress`:
```bash
pipenv install
pipenv shell
```
- Finally run upgrade test in `citus/src/test/regress`:
```bash
pipenv run make check-pg-upgrade old-bindir=<old-bindir> new-bindir=<new-bindir>
```
To see full command list:
```bash
pipenv run citus_tests/upgrade/pg_upgrade_test.py -help
```
How the postgres upgrade test works:
- Temporary folder `tmp_upgrade` is created in `src/test/regress/`, if one exists it is removed first.
- Database is initialized and citus cluster is created(1 coordinator + 2 workers) with old postgres.
- `before_pg_upgrade_schedule` is run with `pg_regress`. This schedule sets up any
objects and data that will be tested for preservation after the upgrade. It
- `after_pg_upgrade_schedule` is run with `pg_regress` to verify that the output
of those tests is the same before the upgrade as after.
- `citus_prepare_pg_upgrade` is run in coordinators and workers.
- Old database is stopped.
- A new database is initialized with new postgres under `tmp_upgrade`.
- Postgres upgrade is performed.
- New database is started in both coordinators and workers.
- `citus_finish_pg_upgrade` is run in coordinators and workers to finalize the upgrade step.
- `after_pg_upgrade_schedule` is run with `pg_regress` to verify that the previously created tables, and data still exist. Router and realtime queries are used to verify this.
### Writing new PG upgrade tests
The main important thing is that we have `upgrade_{name}_before` and
`upgrade_{name}_after` tests. The `before` files are used to setup any objects
and data before the upgrade. The `after` tests shouldn't have any side effects
since they are run twice (once before and once after the upgrade).
Furthermore, anything that is basic Citus functionality should go in the
`upgrade_basic_before`/`upgrade_basic_after` tests. This test file is used
during PG upgrades and Citus upgrades. Any features that don't work in old Citus
versions should thus be added to their own file, because that file will then
only be run during PG versions.
## Citus Upgrade Test
Citus upgrade test is used for testing citus version upgrades from specific version to master. The purpose of this test is to ensure that a newly made change does not result in unexpected upgrade errors.
Currently the citus upgrade test assumes that:
- You have citus artifact tarballs, both for old version and master.
How the citus upgrade test work:
- The script takes `citus-pre-tar` and `citus-post-tar` which should contain citus artifacts.
- It installs the given citus version from `citus-pre-tar`.
- It creates a citus cluster(1 coordinator 2 workers).
- It reports the initial versions.
- It installs the checked out citus version from `citus-post-tar`.
- It restarts the database and runs `ALTER EXTENSION citus UPGRADE`.
- It runs `after_citus_upgrade` schedule to verify that the upgrade is successful.
- It stops the cluster.
Note that when the version of citus changes, we should update `MASTER_VERSION` with the new version of citus otherwise that will be outdated and it will fail.
There is a target for citus upgrade test. We run citus upgrade tests both in normal mode and in mixed mode. In mixed mode, we don't upgrade one of the workers. `'citus.enable_version_checks' : 'false'` is used to prevent citus from giving an error for mixed mode.
To see full command list:
```bash
pipenv run upgrade/citus_upgrade_test.py -help
```
In order to run citus upgrade tests locally you can use:
```bash
pipenv run make check-citus-upgrade-local citus-old-version=v8.0.0
```
For mixed mode:
```bash
pipenv run make check-citus-upgrade-mixed-local citus-old-version=v8.0.0
```

View File

@ -19,25 +19,33 @@ import os
import re
import sys
# https://stackoverflow.com/questions/14132789/relative-imports-for-the-billionth-time/14132912#14132912
sys.path.append(os.path.dirname(os.path.dirname(os.path.abspath(__file__))))
import utils
from utils import USER
from docopt import docopt
from config import (
CitusUpgradeConfig, NODE_PORTS, COORDINATOR_NAME, CITUS_VERSION_SQL, MASTER_VERSION,
NODE_NAMES, USER, WORKER1PORT, MASTER, HOME,
AFTER_CITUS_UPGRADE_COORD_SCHEDULE, BEFORE_CITUS_UPGRADE_COORD_SCHEDULE,
MIXED_AFTER_CITUS_UPGRADE_SCHEDULE, MIXED_BEFORE_CITUS_UPGRADE_SCHEDULE
CitusUpgradeConfig,
CITUS_VERSION_SQL,
MASTER_VERSION,
AFTER_CITUS_UPGRADE_COORD_SCHEDULE,
BEFORE_CITUS_UPGRADE_COORD_SCHEDULE,
MIXED_AFTER_CITUS_UPGRADE_SCHEDULE,
MIXED_BEFORE_CITUS_UPGRADE_SCHEDULE,
)
import upgrade_common as common
import common
def main(config):
install_citus(config.pre_tar_path)
common.initialize_temp_dir(config.temp_dir)
common.initialize_citus_cluster(
config.bindir, config.datadir, config.settings)
config.bindir, config.datadir, config.settings, config
)
report_initial_version(config)
before_upgrade_schedule = get_before_upgrade_schedule(config.mixed_mode)
@ -45,9 +53,9 @@ def main(config):
remove_citus(config.pre_tar_path)
install_citus(config.post_tar_path)
restart_databases(config.bindir, config.datadir, config.mixed_mode)
run_alter_citus(config.bindir, config.mixed_mode)
verify_upgrade(config, config.mixed_mode)
restart_databases(config.bindir, config.datadir, config.mixed_mode, config)
run_alter_citus(config.bindir, config.mixed_mode, config)
verify_upgrade(config, config.mixed_mode, config.node_name_to_ports.values())
after_upgrade_schedule = get_after_upgrade_schedule(config.mixed_mode)
run_test_on_coordinator(config, after_upgrade_schedule)
@ -55,121 +63,150 @@ def main(config):
def install_citus(tar_path):
with utils.cd('/'):
subprocess.run(['tar', 'xvf', tar_path], check=True)
with utils.cd("/"):
subprocess.run(["tar", "xvf", tar_path], check=True)
def report_initial_version(config):
for port in NODE_PORTS.values():
for port in config.node_name_to_ports.values():
actual_citus_version = get_actual_citus_version(config.bindir, port)
print("port:{} citus version {}".format(port, actual_citus_version))
def get_version_number(version):
return re.findall('\d+.\d+', version)[0]
return re.findall("\d+.\d+", version)[0]
def get_actual_citus_version(pg_path, port):
citus_version = utils.psql(pg_path, port, CITUS_VERSION_SQL)
citus_version = citus_version.decode('utf-8')
citus_version = citus_version.decode("utf-8")
return get_version_number(citus_version)
def run_test_on_coordinator(config, schedule):
common.run_pg_regress(config.bindir, config.pg_srcdir,
NODE_PORTS[COORDINATOR_NAME], schedule)
common.run_pg_regress(
config.bindir, config.pg_srcdir, config.coordinator_port(), schedule
)
def remove_citus(tar_path):
with utils.cd('/'):
with utils.cd("/"):
remove_tar_files(tar_path)
def remove_tar_files(tar_path):
ps = subprocess.Popen(('tar', 'tf', tar_path), stdout=subprocess.PIPE)
output = subprocess.check_output(('xargs', 'rm', '-v'), stdin=ps.stdout)
ps = subprocess.Popen(("tar", "tf", tar_path), stdout=subprocess.PIPE)
output = subprocess.check_output(("xargs", "rm", "-v"), stdin=ps.stdout)
ps.wait()
def restart_databases(pg_path, rel_data_path, mixed_mode):
for node_name in NODE_NAMES:
if mixed_mode and NODE_PORTS[node_name] == WORKER1PORT:
def restart_databases(pg_path, rel_data_path, mixed_mode, config):
for node_name in config.node_name_to_ports.keys():
if (
mixed_mode
and config.node_name_to_ports[node_name] == config.chosen_random_worker_port
):
continue
abs_data_path = os.path.abspath(os.path.join(rel_data_path, node_name))
restart_database(
pg_path=pg_path, abs_data_path=abs_data_path, node_name=node_name)
pg_path=pg_path,
abs_data_path=abs_data_path,
node_name=node_name,
node_ports=config.node_name_to_ports,
logfile_prefix=config.name,
)
def restart_database(pg_path, abs_data_path, node_name):
def restart_database(pg_path, abs_data_path, node_name, node_ports, logfile_prefix):
command = [
os.path.join(pg_path, 'pg_ctl'), 'restart',
'--pgdata', abs_data_path,
'-U', USER,
'-o', '-p {}'.format(NODE_PORTS[node_name]),
'--log', os.path.join(abs_data_path, 'logfile_' + node_name)
os.path.join(pg_path, "pg_ctl"),
"restart",
"--pgdata",
abs_data_path,
"-U",
USER,
"-o",
"-p {}".format(node_ports[node_name]),
"--log",
os.path.join(abs_data_path, common.logfile_name(logfile_prefix, node_name)),
]
subprocess.run(command, check=True)
def run_alter_citus(pg_path, mixed_mode):
for port in NODE_PORTS.values():
if mixed_mode and port == WORKER1PORT:
def run_alter_citus(pg_path, mixed_mode, config):
for port in config.node_name_to_ports.values():
if mixed_mode and port == config.chosen_random_worker_port:
continue
utils.psql(pg_path, port, "ALTER EXTENSION citus UPDATE;")
def verify_upgrade(config, mixed_mode):
for port in NODE_PORTS.values():
def verify_upgrade(config, mixed_mode, node_ports):
for port in node_ports:
actual_citus_version = get_actual_citus_version(config.bindir, port)
expected_citus_version = MASTER_VERSION
if expected_citus_version != actual_citus_version and not (mixed_mode and port == WORKER1PORT):
print("port: {} citus version {} expected {}".format(
port, actual_citus_version, expected_citus_version))
if expected_citus_version != actual_citus_version and not (
mixed_mode and port == config.chosen_random_worker_port
):
print(
"port: {} citus version {} expected {}".format(
port, actual_citus_version, expected_citus_version
)
)
sys.exit(1)
else:
print("port:{} citus version {}".format(port, actual_citus_version))
def get_before_upgrade_schedule(mixed_mode):
if mixed_mode:
return MIXED_BEFORE_CITUS_UPGRADE_SCHEDULE
else:
return BEFORE_CITUS_UPGRADE_COORD_SCHEDULE
def get_after_upgrade_schedule(mixed_mode):
if mixed_mode:
return MIXED_AFTER_CITUS_UPGRADE_SCHEDULE
else:
return AFTER_CITUS_UPGRADE_COORD_SCHEDULE
# IsRunningOnLocalMachine returns true if the upgrade test is run on
# local machine, in which case the old citus version will be installed
# and it will be upgraded to the current code.
def IsRunningOnLocalMachine(arguments):
return arguments['--citus-old-version']
return arguments["--citus-old-version"]
def generate_citus_tarballs(citus_version):
tmp_dir = 'tmp_citus_tarballs'
citus_old_tarpath = os.path.abspath(os.path.join(
tmp_dir, 'install-citus{}.tar'.format(citus_version)))
citus_new_tarpath = os.path.abspath(os.path.join(tmp_dir, 'install-citusmaster.tar'))
tmp_dir = "tmp_citus_tarballs"
citus_old_tarpath = os.path.abspath(
os.path.join(tmp_dir, "install-citus{}.tar".format(citus_version))
)
citus_new_tarpath = os.path.abspath(
os.path.join(tmp_dir, "install-citusmaster.tar")
)
common.initialize_temp_dir_if_not_exists(tmp_dir)
local_script_path = os.path.abspath('upgrade/generate_citus_tarballs.sh')
local_script_path = os.path.abspath("upgrade/generate_citus_tarballs.sh")
with utils.cd(tmp_dir):
subprocess.check_call([
local_script_path, citus_version
])
subprocess.check_call([local_script_path, citus_version])
return [citus_old_tarpath, citus_new_tarpath]
if __name__ == '__main__':
args = docopt(__doc__, version='citus_upgrade_test')
if __name__ == "__main__":
args = docopt(__doc__, version="citus_upgrade_test")
if IsRunningOnLocalMachine(args):
citus_tarball_paths = generate_citus_tarballs(
args['--citus-old-version'])
args['--citus-pre-tar'] = citus_tarball_paths[0]
args['--citus-post-tar'] = citus_tarball_paths[1]
citus_tarball_paths = generate_citus_tarballs(args["--citus-old-version"])
args["--citus-pre-tar"] = citus_tarball_paths[0]
args["--citus-post-tar"] = citus_tarball_paths[1]
config = CitusUpgradeConfig(args)
atexit.register(common.stop_databases, config.bindir, config.datadir)
atexit.register(
common.stop_databases,
config.bindir,
config.datadir,
config.node_name_to_ports,
config.name,
)
main(config)

View File

@ -0,0 +1,137 @@
#!/usr/bin/env python3
"""upgrade_test
Usage:
upgrade_test --old-bindir=<old-bindir> --new-bindir=<new-bindir> --pgxsdir=<pgxsdir>
Options:
--old-bindir=<old-bindir> The old PostgreSQL executable directory(ex: '~/.pgenv/pgsql-10.4/bin')
--new-bindir=<new-bindir> The new PostgreSQL executable directory(ex: '~/.pgenv/pgsql-11.3/bin')
--pgxsdir=<pgxsdir> Path to the PGXS directory(ex: ~/.pgenv/src/postgresql-11.3)
"""
import sys, os
# https://stackoverflow.com/questions/14132789/relative-imports-for-the-billionth-time/14132912#14132912
sys.path.append(os.path.dirname(os.path.dirname(os.path.abspath(__file__))))
from config import (
PGUpgradeConfig,
AFTER_PG_UPGRADE_SCHEDULE,
BEFORE_PG_UPGRADE_SCHEDULE,
)
from docopt import docopt
import utils
from utils import USER
import atexit
import subprocess
import common
def citus_prepare_pg_upgrade(pg_path, node_ports):
for port in node_ports:
utils.psql(pg_path, port, "SELECT citus_prepare_pg_upgrade();")
def perform_postgres_upgrade(
old_bindir, new_bindir, old_datadir, new_datadir, node_names
):
for node_name in node_names:
base_new_data_path = os.path.abspath(new_datadir)
base_old_data_path = os.path.abspath(old_datadir)
with utils.cd(base_new_data_path):
abs_new_data_path = os.path.join(base_new_data_path, node_name)
abs_old_data_path = os.path.join(base_old_data_path, node_name)
command = [
os.path.join(new_bindir, "pg_upgrade"),
"--username",
USER,
"--old-bindir",
old_bindir,
"--new-bindir",
new_bindir,
"--old-datadir",
abs_old_data_path,
"--new-datadir",
abs_new_data_path,
]
subprocess.run(command, check=True)
def citus_finish_pg_upgrade(pg_path, node_ports):
for port in node_ports:
utils.psql(pg_path, port, "SELECT citus_finish_pg_upgrade();")
def stop_all_databases(old_bindir, new_bindir, old_datadir, new_datadir, config):
common.stop_databases(
old_bindir, old_datadir, config.node_name_to_ports, config.name
)
common.stop_databases(
new_bindir, new_datadir, config.node_name_to_ports, config.name
)
def main(config):
common.initialize_temp_dir(config.temp_dir)
common.initialize_citus_cluster(
config.old_bindir, config.old_datadir, config.settings, config
)
common.run_pg_regress(
config.old_bindir,
config.pg_srcdir,
config.coordinator_port(),
BEFORE_PG_UPGRADE_SCHEDULE,
)
common.run_pg_regress(
config.old_bindir,
config.pg_srcdir,
config.coordinator_port(),
AFTER_PG_UPGRADE_SCHEDULE,
)
citus_prepare_pg_upgrade(config.old_bindir, config.node_name_to_ports.values())
# prepare should be idempotent, calling it a second time should never fail.
citus_prepare_pg_upgrade(config.old_bindir, config.node_name_to_ports.values())
common.stop_databases(
config.old_bindir, config.old_datadir, config.node_name_to_ports, config.name
)
common.initialize_db_for_cluster(
config.new_bindir,
config.new_datadir,
config.settings,
config.node_name_to_ports.keys(),
)
perform_postgres_upgrade(
config.old_bindir,
config.new_bindir,
config.old_datadir,
config.new_datadir,
config.node_name_to_ports.keys(),
)
common.start_databases(
config.new_bindir, config.new_datadir, config.node_name_to_ports, config.name
)
citus_finish_pg_upgrade(config.new_bindir, config.node_name_to_ports.values())
common.run_pg_regress(
config.new_bindir,
config.pg_srcdir,
config.coordinator_port(),
AFTER_PG_UPGRADE_SCHEDULE,
)
if __name__ == "__main__":
config = PGUpgradeConfig(docopt(__doc__, version="upgrade_test"))
atexit.register(
stop_all_databases,
config.old_bindir,
config.new_bindir,
config.old_datadir,
config.new_datadir,
config,
)
main(config)

View File

@ -1,16 +1,16 @@
import subprocess
import os
from config import USER
USER = "postgres"
def psql(pg_path, port, command):
return subprocess.check_output([
os.path.join(pg_path, 'psql'),
'-U', USER,
'-p', str(port),
'-c', command]
return subprocess.check_output(
[os.path.join(pg_path, "psql"), "-U", USER, "-p", str(port), "-c", command],
)
# Taken from https://stackoverflow.com/questions/431684/how-do-i-change-directory-cd-in-python/13197763#13197763
@ -22,6 +22,7 @@ class cd(object):
def __enter__(self):
self.savedPath = os.getcwd()
print(self.savedPath)
os.chdir(self.newPath)
def __exit__(self, etype, value, traceback):

View File

@ -0,0 +1,3 @@
test: create_test intermediate_result_pruning_create
test: prepared_statements_create_load ch_benchmarks_create_load
test: dropped_columns_create_load distributed_planning_create_load

View File

@ -0,0 +1,236 @@
SET search_path to "ch benchmarks";
SET search_path to "ch benchmarks";
-- Query 1
SELECT
ol_number,
sum(ol_quantity) as sum_qty,
sum(ol_amount) as sum_amount,
avg(ol_quantity) as avg_qty,
avg(ol_amount) as avg_amount,
count(*) as count_order
FROM order_line
WHERE ol_delivery_d > '2007-01-02 00:00:00.000000'
GROUP BY ol_number
ORDER BY ol_number;
ol_number | sum_qty | sum_amount | avg_qty | avg_amount | count_order
---------------------------------------------------------------------
0 | 0 | 0.00 | 0.00000000000000000000 | 0.00000000000000000000 | 1
1 | 1 | 1.00 | 1.00000000000000000000 | 1.00000000000000000000 | 1
2 | 2 | 2.00 | 2.0000000000000000 | 2.0000000000000000 | 1
3 | 3 | 3.00 | 3.0000000000000000 | 3.0000000000000000 | 1
4 | 4 | 4.00 | 4.0000000000000000 | 4.0000000000000000 | 1
5 | 5 | 5.00 | 5.0000000000000000 | 5.0000000000000000 | 1
6 | 6 | 6.00 | 6.0000000000000000 | 6.0000000000000000 | 1
7 | 7 | 7.00 | 7.0000000000000000 | 7.0000000000000000 | 1
8 | 8 | 8.00 | 8.0000000000000000 | 8.0000000000000000 | 1
9 | 9 | 9.00 | 9.0000000000000000 | 9.0000000000000000 | 1
10 | 10 | 10.00 | 10.0000000000000000 | 10.0000000000000000 | 1
(11 rows)
-- Query 2
SELECT
su_suppkey,
su_name,
n_name,
i_id,
i_name,
su_address,
su_phone,
su_comment
FROM
item,
supplier,
stock,
nation,
region,
(SELECT
s_i_id AS m_i_id,
min(s_quantity) as m_s_quantity
FROM
stock,
supplier,
nation,
region
WHERE mod((s_w_id*s_i_id),10000)=su_suppkey
AND su_nationkey=n_nationkey
AND n_regionkey=r_regionkey
AND r_name LIKE 'Europ%'
GROUP BY s_i_id) m
WHERE i_id = s_i_id
AND mod((s_w_id * s_i_id), 10000) = su_suppkey
AND su_nationkey = n_nationkey
AND n_regionkey = r_regionkey
AND i_data LIKE '%b'
AND r_name LIKE 'Europ%'
AND i_id = m_i_id
AND s_quantity = m_s_quantity
ORDER BY
n_name,
su_name,
i_id;
su_suppkey | su_name | n_name | i_id | i_name | su_address | su_phone | su_comment
---------------------------------------------------------------------
9 | abc | Germany | 3 | Keyboard | def | ghi | jkl
4 | abc | The Netherlands | 2 | Keyboard | def | ghi | jkl
(2 rows)
-- Query 3
SELECT
ol_o_id,
ol_w_id,
ol_d_id,
sum(ol_amount) AS revenue,
o_entry_d
FROM
customer,
new_order,
oorder,
order_line
WHERE c_state LIKE 'C%' -- used to ba A%, but C% works with our small data
AND c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND no_w_id = o_w_id
AND no_d_id = o_d_id
AND no_o_id = o_id
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND o_entry_d > '2007-01-02 00:00:00.000000'
GROUP BY
ol_o_id,
ol_w_id,
ol_d_id,
o_entry_d
ORDER BY
revenue DESC,
o_entry_d;
ol_o_id | ol_w_id | ol_d_id | revenue | o_entry_d
---------------------------------------------------------------------
10 | 10 | 10 | 10.00 | Fri Oct 17 00:00:00 2008
9 | 9 | 9 | 9.00 | Fri Oct 17 00:00:00 2008
8 | 8 | 8 | 8.00 | Fri Oct 17 00:00:00 2008
7 | 7 | 7 | 7.00 | Fri Oct 17 00:00:00 2008
6 | 6 | 6 | 6.00 | Fri Oct 17 00:00:00 2008
5 | 5 | 5 | 5.00 | Fri Oct 17 00:00:00 2008
4 | 4 | 4 | 4.00 | Fri Oct 17 00:00:00 2008
3 | 3 | 3 | 3.00 | Fri Oct 17 00:00:00 2008
2 | 2 | 2 | 2.00 | Fri Oct 17 00:00:00 2008
1 | 1 | 1 | 1.00 | Fri Oct 17 00:00:00 2008
0 | 0 | 0 | 0.00 | Fri Oct 17 00:00:00 2008
(11 rows)
-- Query 4
SELECT
o_ol_cnt,
count(*) as order_count
FROM
oorder
WHERE o_entry_d >= '2007-01-02 00:00:00.000000'
AND o_entry_d < '2012-01-02 00:00:00.000000'
AND exists (SELECT *
FROM order_line
WHERE o_id = ol_o_id
AND o_w_id = ol_w_id
AND o_d_id = ol_d_id
AND ol_delivery_d >= o_entry_d)
GROUP BY o_ol_cnt
ORDER BY o_ol_cnt;
o_ol_cnt | order_count
---------------------------------------------------------------------
1 | 11
(1 row)
-- Query 5
SELECT
n_name,
sum(ol_amount) AS revenue
FROM
customer,
oorder,
order_line,
stock,
supplier,
nation,
region
WHERE c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND ol_o_id = o_id
AND ol_w_id = o_w_id
AND ol_d_id=o_d_id
AND ol_w_id = s_w_id
AND ol_i_id = s_i_id
AND mod((s_w_id * s_i_id),10000) = su_suppkey
-- our dataset does not have the supplier in the same nation as the customer causing this
-- join to filter out all the data. We verify later on that we can actually perform an
-- ascii(substr(c_state,1,1)) == reference table column join later on so it should not
-- matter we skip this filter here.
--AND ascii(substr(c_state,1,1)) = su_nationkey
AND su_nationkey = n_nationkey
AND n_regionkey = r_regionkey
AND r_name = 'Europe'
AND o_entry_d >= '2007-01-02 00:00:00.000000'
GROUP BY n_name
ORDER BY revenue DESC;
n_name | revenue
---------------------------------------------------------------------
Germany | 3.00
The Netherlands | 2.00
(2 rows)
-- Query 6
SELECT
sum(ol_amount) AS revenue
FROM order_line
WHERE ol_delivery_d >= '1999-01-01 00:00:00.000000'
AND ol_delivery_d < '2020-01-01 00:00:00.000000'
AND ol_quantity BETWEEN 1 AND 100000;
revenue
---------------------------------------------------------------------
55.00
(1 row)
-- Query 7
SELECT
su_nationkey as supp_nation,
substr(c_state,1,1) as cust_nation,
extract(year from o_entry_d) as l_year,
sum(ol_amount) as revenue
FROM
supplier,
stock,
order_line,
oorder,
customer,
nation n1,
nation n2
WHERE ol_supply_w_id = s_w_id
AND ol_i_id = s_i_id
AND mod((s_w_id * s_i_id), 10000) = su_suppkey
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND su_nationkey = n1.n_nationkey
AND ascii(substr(c_state,1,1)) = n2.n_nationkey
AND (
(n1.n_name = 'Germany' AND n2.n_name = 'Cambodia')
OR (n1.n_name = 'Cambodia' AND n2.n_name = 'Germany')
)
AND ol_delivery_d BETWEEN '2007-01-02 00:00:00.000000' AND '2012-01-02 00:00:00.000000'
GROUP BY
su_nationkey,
substr(c_state,1,1),
extract(year from o_entry_d)
ORDER BY
su_nationkey,
cust_nation,
l_year;
supp_nation | cust_nation | l_year | revenue
---------------------------------------------------------------------
9 | C | 2008 | 3.00
(1 row)

View File

@ -0,0 +1,232 @@
SET search_path to "ch benchmarks";
-- Query 8
SELECT
extract(year from o_entry_d) as l_year,
sum(case when n2.n_name = 'Germany' then ol_amount else 0 end) / sum(ol_amount) as mkt_share
FROM
item,
supplier,
stock,
order_line,
oorder,
customer,
nation n1,
nation n2,
region
WHERE i_id = s_i_id
AND ol_i_id = s_i_id
AND ol_supply_w_id = s_w_id
AND mod((s_w_id * s_i_id),10000) = su_suppkey
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND n1.n_nationkey = ascii(substr(c_state,1,1))
AND n1.n_regionkey = r_regionkey
AND ol_i_id < 1000
AND r_name = 'Europe'
AND su_nationkey = n2.n_nationkey
AND o_entry_d BETWEEN '2007-01-02 00:00:00.000000' AND '2012-01-02 00:00:00.000000'
AND i_data LIKE '%b'
AND i_id = ol_i_id
GROUP BY extract(YEAR FROM o_entry_d)
ORDER BY l_year;
l_year | mkt_share
---------------------------------------------------------------------
2008 | 0.50000000000000000000
(1 row)
-- Query 9
SELECT
n_name,
extract(year from o_entry_d) as l_year,
sum(ol_amount) as sum_profit
FROM
item,
stock,
supplier,
order_line,
oorder,
nation
WHERE ol_i_id = s_i_id
AND ol_supply_w_id = s_w_id
AND mod((s_w_id * s_i_id), 10000) = su_suppkey
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND ol_i_id = i_id
AND su_nationkey = n_nationkey
AND i_data LIKE '%b' -- this used to be %BB but that will not work with our small dataset
GROUP BY
n_name,
extract(YEAR FROM o_entry_d)
ORDER BY
n_name,
l_year DESC;
n_name | l_year | sum_profit
---------------------------------------------------------------------
Germany | 2008 | 3.00
The Netherlands | 2008 | 2.00
United States | 2008 | 1.00
(3 rows)
-- Query 10
SELECT
c_id,
c_last,
sum(ol_amount) AS revenue,
c_city,
c_phone,
n_name
FROM
customer,
oorder,
order_line,
nation
WHERE c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND o_entry_d >= '2007-01-02 00:00:00.000000'
AND o_entry_d <= ol_delivery_d
AND n_nationkey = ascii(substr(c_state,1,1))
GROUP BY
c_id,
c_last,
c_city,
c_phone,
n_name
ORDER BY revenue DESC;
c_id | c_last | revenue | c_city | c_phone | n_name
---------------------------------------------------------------------
10 | John | 10.00 | Some City | +1 000 0000000 | Cambodia
9 | John | 9.00 | Some City | +1 000 0000000 | Cambodia
8 | John | 8.00 | Some City | +1 000 0000000 | Cambodia
7 | John | 7.00 | Some City | +1 000 0000000 | Cambodia
6 | John | 6.00 | Some City | +1 000 0000000 | Cambodia
5 | John | 5.00 | Some City | +1 000 0000000 | Cambodia
4 | John | 4.00 | Some City | +1 000 0000000 | Cambodia
3 | John | 3.00 | Some City | +1 000 0000000 | Cambodia
2 | John | 2.00 | Some City | +1 000 0000000 | Cambodia
1 | John | 1.00 | Some City | +1 000 0000000 | Cambodia
0 | John | 0.00 | Some City | +1 000 0000000 | Cambodia
(11 rows)
-- Query 11
SELECT
s_i_id,
sum(s_order_cnt) AS ordercount
FROM
stock,
supplier,
nation
WHERE mod((s_w_id * s_i_id),10000) = su_suppkey
AND su_nationkey = n_nationkey
AND n_name = 'Germany'
GROUP BY s_i_id
HAVING sum(s_order_cnt) >
(SELECT sum(s_order_cnt) * .005
FROM
stock,
supplier,
nation
WHERE mod((s_w_id * s_i_id),10000) = su_suppkey
AND su_nationkey = n_nationkey
AND n_name = 'Germany')
ORDER BY ordercount DESC;
s_i_id | ordercount
---------------------------------------------------------------------
3 | 3
(1 row)
-- Query 12
SELECT
o_ol_cnt,
sum(case when o_carrier_id = 1 or o_carrier_id = 2 then 1 else 0 end) as high_line_count,
sum(case when o_carrier_id <> 1 and o_carrier_id <> 2 then 1 else 0 end) as low_line_count
FROM
oorder,
order_line
WHERE ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND o_entry_d <= ol_delivery_d
AND ol_delivery_d < '2020-01-01 00:00:00.000000'
GROUP BY o_ol_cnt
ORDER BY o_ol_cnt;
o_ol_cnt | high_line_count | low_line_count
---------------------------------------------------------------------
1 | 2 | 9
(1 row)
-- Query 13
SELECT
c_count,
count(*) AS custdist
FROM (SELECT
c_id,
count(o_id)
FROM customer
LEFT OUTER JOIN oorder ON (
c_w_id = o_w_id
AND c_d_id = o_d_id
AND c_id = o_c_id
AND o_carrier_id > 8)
GROUP BY c_id) AS c_orders (c_id, c_count)
GROUP BY c_count
ORDER BY
custdist DESC,
c_count DESC;
c_count | custdist
---------------------------------------------------------------------
0 | 9
1 | 2
(2 rows)
-- Query 14
SELECT
100.00 * sum(CASE WHEN i_data LIKE 'PR%' THEN ol_amount ELSE 0 END) / (1+sum(ol_amount)) AS promo_revenue
FROM
order_line,
item
WHERE ol_i_id = i_id
AND ol_delivery_d >= '2007-01-02 00:00:00.000000'
AND ol_delivery_d < '2020-01-02 00:00:00.000000';
promo_revenue
---------------------------------------------------------------------
0.00000000000000000000
(1 row)
-- Query 15
WITH revenue (supplier_no, total_revenue) AS (
SELECT
mod((s_w_id * s_i_id),10000) AS supplier_no,
sum(ol_amount) AS total_revenue
FROM
order_line,
stock
WHERE ol_i_id = s_i_id
AND ol_supply_w_id = s_w_id
AND ol_delivery_d >= '2007-01-02 00:00:00.000000'
GROUP BY mod((s_w_id * s_i_id),10000))
SELECT
su_suppkey,
su_name,
su_address,
su_phone,
total_revenue
FROM
supplier,
revenue
WHERE su_suppkey = supplier_no
AND total_revenue = (SELECT max(total_revenue) FROM revenue)
ORDER BY su_suppkey;
su_suppkey | su_name | su_address | su_phone | total_revenue
---------------------------------------------------------------------
9 | abc | def | ghi | 3.00
(1 row)

View File

@ -0,0 +1,36 @@
SET search_path to "ch benchmarks";
-- Subquery + repartion is supported when it is an IN query where the subquery
-- returns unique results (because it's converted to an INNER JOIN)
select s_i_id
from stock, order_line
where
s_i_id in (select i_id from item)
AND s_i_id = ol_i_id
order by s_i_id;
s_i_id
---------------------------------------------------------------------
1
2
3
(3 rows)
select su_name, su_address
from supplier, nation
where su_suppkey in
(select mod(s_i_id * s_w_id, 10000)
from stock, order_line
where s_i_id in
(select i_id
from item
where i_data like 'ab%')
and ol_i_id=s_i_id
and ol_delivery_d > '2010-05-23 12:00:00'
group by s_i_id, s_w_id, s_quantity
having 2*s_quantity > sum(ol_quantity))
and su_nationkey = n_nationkey
and n_name = 'Germany'
order by su_name;
su_name | su_address
---------------------------------------------------------------------
(0 rows)

View File

@ -0,0 +1,135 @@
SET search_path to "ch benchmarks";
--Q16
SELECT
i_name,
substr(i_data, 1, 3) AS brand,
i_price,
count(DISTINCT (mod((s_w_id * s_i_id),10000))) AS supplier_cnt
FROM
stock,
item
WHERE i_id = s_i_id
AND i_data NOT LIKE 'zz%'
AND (mod((s_w_id * s_i_id),10000) NOT IN
(SELECT su_suppkey
FROM supplier
WHERE su_comment LIKE '%bad%'))
GROUP BY
i_name,
substr(i_data, 1, 3),
i_price
ORDER BY supplier_cnt DESC;
i_name | brand | i_price | supplier_cnt
---------------------------------------------------------------------
Keyboard | co | 50.00 | 3
(1 row)
--Q17
SELECT
sum(ol_amount) / 2.0 AS avg_yearly
FROM
order_line,
(SELECT
i_id,
avg(ol_quantity) AS a
FROM
item,
order_line
WHERE i_data LIKE '%b'
AND ol_i_id = i_id
GROUP BY i_id) t
WHERE ol_i_id = t.i_id;
avg_yearly
---------------------------------------------------------------------
27.5000000000000000
(1 row)
-- this filter was at the end causing the dataset to be empty. it should not have any
-- influence on how the query gets planned so I removed the clause
--AND ol_quantity < t.a;
-- Query 18
SELECT
c_last,
c_id o_id,
o_entry_d,
o_ol_cnt,
sum(ol_amount)
FROM
customer,
oorder,
order_line
WHERE c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
GROUP BY
o_id,
o_w_id,
o_d_id,
c_id,
c_last,
o_entry_d,
o_ol_cnt
HAVING sum(ol_amount) > 5 -- was 200, but thats too big for the dataset
ORDER BY
sum(ol_amount) DESC,
o_entry_d;
c_last | o_id | o_entry_d | o_ol_cnt | sum
---------------------------------------------------------------------
John | 10 | Fri Oct 17 00:00:00 2008 | 1 | 10.00
John | 9 | Fri Oct 17 00:00:00 2008 | 1 | 9.00
John | 8 | Fri Oct 17 00:00:00 2008 | 1 | 8.00
John | 7 | Fri Oct 17 00:00:00 2008 | 1 | 7.00
John | 6 | Fri Oct 17 00:00:00 2008 | 1 | 6.00
(5 rows)
-- Query 19
SELECT
sum(ol_amount) AS revenue
FROM
order_line,
item
WHERE ( ol_i_id = i_id
AND i_data LIKE '%a'
AND ol_quantity >= 1
AND ol_quantity <= 10
AND i_price BETWEEN 1 AND 400000
AND ol_w_id IN (1,2,3))
OR ( ol_i_id = i_id
AND i_data LIKE '%b'
AND ol_quantity >= 1
AND ol_quantity <= 10
AND i_price BETWEEN 1 AND 400000
AND ol_w_id IN (1,2,4))
OR ( ol_i_id = i_id
AND i_data LIKE '%c'
AND ol_quantity >= 1
AND ol_quantity <= 10
AND i_price BETWEEN 1 AND 400000
AND ol_w_id IN (1,5,3));
revenue
---------------------------------------------------------------------
7.00
(1 row)
select su_name, su_address
from supplier, nation
where su_suppkey in
(select mod(s_i_id * s_w_id, 10000)
from stock, order_line
where s_i_id in
(select i_id
from item)
and ol_i_id=s_i_id
and ol_delivery_d > '2010-05-23 12:00:00'
group by s_i_id, s_w_id, s_quantity
having 2*s_quantity > sum(ol_quantity))
and su_nationkey = n_nationkey
and n_name = 'GERMANY'
order by su_name;
su_name | su_address
---------------------------------------------------------------------
(0 rows)

View File

@ -0,0 +1,58 @@
SET search_path to "ch benchmarks";
-- Query 21
-- DATA SET DOES NOT COVER THIS QUERY
SELECT
su_name,
count(*) AS numwait
FROM
supplier,
order_line l1,
oorder,
stock,
nation
WHERE ol_o_id = o_id
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_w_id = s_w_id
AND ol_i_id = s_i_id
AND mod((s_w_id * s_i_id),10000) = su_suppkey
AND l1.ol_delivery_d > o_entry_d
AND NOT exists (SELECT *
FROM order_line l2
WHERE l2.ol_o_id = l1.ol_o_id
AND l2.ol_w_id = l1.ol_w_id
AND l2.ol_d_id = l1.ol_d_id
AND l2.ol_delivery_d > l1.ol_delivery_d)
AND su_nationkey = n_nationkey
AND n_name = 'Germany'
GROUP BY su_name
ORDER BY
numwait desc,
su_name;
su_name | numwait
---------------------------------------------------------------------
(0 rows)
-- Query 22
-- DATA SET DOES NOT COVER THIS QUERY
SELECT
substr(c_state,1,1) AS country,
count(*) AS numcust,
sum(c_balance) AS totacctbal
FROM customer
WHERE substr(c_phone,1,1) in ('1','2','3','4','5','6','7')
AND c_balance > (SELECT avg(c_BALANCE)
FROM customer
WHERE c_balance > 0.00
AND substr(c_phone,1,1) in ('1','2','3','4','5','6','7'))
AND NOT exists (SELECT *
FROM oorder
WHERE o_c_id = c_id
AND o_w_id = c_w_id
AND o_d_id = c_d_id)
GROUP BY substr(c_state,1,1)
ORDER BY substr(c_state,1,1);
country | numcust | totacctbal
---------------------------------------------------------------------
(0 rows)

View File

@ -0,0 +1,207 @@
SET search_path to "ch benchmarks";
-- Query 20
SELECT
su_name,
su_address
FROM
supplier,
nation
WHERE su_suppkey in
(SELECT
mod(s_i_id * s_w_id, 10000)
FROM
stock,
order_line
WHERE s_i_id IN
(SELECT i_id
FROM item
WHERE i_data LIKE 'co%')
AND ol_i_id = s_i_id
AND ol_delivery_d > '2008-05-23 12:00:00' -- was 2010, but our order is in 2008
GROUP BY s_i_id, s_w_id, s_quantity
HAVING 2*s_quantity > sum(ol_quantity))
AND su_nationkey = n_nationkey
AND n_name = 'Germany'
ORDER BY su_name;
su_name | su_address
---------------------------------------------------------------------
abc | def
(1 row)
-- Multiple subqueries are supported IN and a NOT IN when no repartition join
-- is necessary and the IN subquery returns unique results
select s_i_id
from stock
where
s_i_id in (select i_id from item)
AND s_i_id not in (select i_im_id from item);
s_i_id
---------------------------------------------------------------------
(0 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
ORDER BY s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select true)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select true)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having (select true)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having (select false)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
(0 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select true)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select false)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
(0 rows)
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select true)
order by s_i_id;
s_i_id | ordercount
---------------------------------------------------------------------
1 | 1
2 | 2
3 | 3
5000 | 6
(4 rows)

View File

@ -0,0 +1,233 @@
CREATE SCHEMA "ch benchmarks";
SET search_path to "ch benchmarks";
CREATE TABLE order_line (
ol_w_id int NOT NULL,
ol_d_id int NOT NULL,
ol_o_id int NOT NULL,
ol_number int NOT NULL,
ol_i_id int NOT NULL,
ol_delivery_d timestamp NULL DEFAULT NULL,
ol_amount decimal(6,2) NOT NULL,
ol_supply_w_id int NOT NULL,
ol_quantity decimal(2,0) NOT NULL,
ol_dist_info char(24) NOT NULL,
PRIMARY KEY (ol_w_id,ol_d_id,ol_o_id,ol_number)
);
CREATE TABLE new_order (
no_w_id int NOT NULL,
no_d_id int NOT NULL,
no_o_id int NOT NULL,
PRIMARY KEY (no_w_id,no_d_id,no_o_id)
);
CREATE TABLE stock (
s_w_id int NOT NULL,
s_i_id int NOT NULL,
s_quantity decimal(4,0) NOT NULL,
s_ytd decimal(8,2) NOT NULL,
s_order_cnt int NOT NULL,
s_remote_cnt int NOT NULL,
s_data varchar(50) NOT NULL,
s_dist_01 char(24) NOT NULL,
s_dist_02 char(24) NOT NULL,
s_dist_03 char(24) NOT NULL,
s_dist_04 char(24) NOT NULL,
s_dist_05 char(24) NOT NULL,
s_dist_06 char(24) NOT NULL,
s_dist_07 char(24) NOT NULL,
s_dist_08 char(24) NOT NULL,
s_dist_09 char(24) NOT NULL,
s_dist_10 char(24) NOT NULL,
PRIMARY KEY (s_w_id,s_i_id)
);
CREATE TABLE oorder (
o_w_id int NOT NULL,
o_d_id int NOT NULL,
o_id int NOT NULL,
o_c_id int NOT NULL,
o_carrier_id int DEFAULT NULL,
o_ol_cnt decimal(2,0) NOT NULL,
o_all_local decimal(1,0) NOT NULL,
o_entry_d timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
PRIMARY KEY (o_w_id,o_d_id,o_id),
UNIQUE (o_w_id,o_d_id,o_c_id,o_id)
);
CREATE TABLE history (
h_c_id int NOT NULL,
h_c_d_id int NOT NULL,
h_c_w_id int NOT NULL,
h_d_id int NOT NULL,
h_w_id int NOT NULL,
h_date timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
h_amount decimal(6,2) NOT NULL,
h_data varchar(24) NOT NULL
);
CREATE TABLE customer (
c_w_id int NOT NULL,
c_d_id int NOT NULL,
c_id int NOT NULL,
c_discount decimal(4,4) NOT NULL,
c_credit char(2) NOT NULL,
c_last varchar(16) NOT NULL,
c_first varchar(16) NOT NULL,
c_credit_lim decimal(12,2) NOT NULL,
c_balance decimal(12,2) NOT NULL,
c_ytd_payment float NOT NULL,
c_payment_cnt int NOT NULL,
c_delivery_cnt int NOT NULL,
c_street_1 varchar(20) NOT NULL,
c_street_2 varchar(20) NOT NULL,
c_city varchar(20) NOT NULL,
c_state char(2) NOT NULL,
c_zip char(9) NOT NULL,
c_phone char(16) NOT NULL,
c_since timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
c_middle char(2) NOT NULL,
c_data varchar(500) NOT NULL,
PRIMARY KEY (c_w_id,c_d_id,c_id)
);
CREATE TABLE district (
d_w_id int NOT NULL,
d_id int NOT NULL,
d_ytd decimal(12,2) NOT NULL,
d_tax decimal(4,4) NOT NULL,
d_next_o_id int NOT NULL,
d_name varchar(10) NOT NULL,
d_street_1 varchar(20) NOT NULL,
d_street_2 varchar(20) NOT NULL,
d_city varchar(20) NOT NULL,
d_state char(2) NOT NULL,
d_zip char(9) NOT NULL,
PRIMARY KEY (d_w_id,d_id)
);
CREATE TABLE item (
i_id int NOT NULL,
i_name varchar(24) NOT NULL,
i_price decimal(5,2) NOT NULL,
i_data varchar(50) NOT NULL,
i_im_id int NOT NULL,
PRIMARY KEY (i_id)
);
CREATE TABLE warehouse (
w_id int NOT NULL,
w_ytd decimal(12,2) NOT NULL,
w_tax decimal(4,4) NOT NULL,
w_name varchar(10) NOT NULL,
w_street_1 varchar(20) NOT NULL,
w_street_2 varchar(20) NOT NULL,
w_city varchar(20) NOT NULL,
w_state char(2) NOT NULL,
w_zip char(9) NOT NULL,
PRIMARY KEY (w_id)
);
CREATE TABLE region (
r_regionkey int not null,
r_name char(55) not null,
r_comment char(152) not null,
PRIMARY KEY ( r_regionkey )
);
CREATE TABLE nation (
n_nationkey int not null,
n_name char(25) not null,
n_regionkey int not null,
n_comment char(152) not null,
PRIMARY KEY ( n_nationkey )
);
CREATE TABLE supplier (
su_suppkey int not null,
su_name char(25) not null,
su_address varchar(40) not null,
su_nationkey int not null,
su_phone char(15) not null,
su_acctbal numeric(12,2) not null,
su_comment char(101) not null,
PRIMARY KEY ( su_suppkey )
);
SELECT create_distributed_table('order_line','ol_w_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('new_order','no_w_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('stock','s_w_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('oorder','o_w_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('history','h_w_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('customer','c_w_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('district','d_w_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('warehouse','w_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_reference_table('item');
create_reference_table
---------------------------------------------------------------------
(1 row)
SELECT create_reference_table('region');
create_reference_table
---------------------------------------------------------------------
(1 row)
SELECT create_reference_table('nation');
create_reference_table
---------------------------------------------------------------------
(1 row)
SELECT create_reference_table('supplier');
create_reference_table
---------------------------------------------------------------------
(1 row)
TRUNCATE order_line, new_order, stock, oorder, history, customer, district, warehouse, item, region, nation, supplier; -- for easy copy in development
INSERT INTO supplier SELECT c, 'abc', 'def', c, 'ghi', c, 'jkl' FROM generate_series(0,10) AS c;
INSERT INTO new_order SELECT c, c, c FROM generate_series(0,10) AS c;
INSERT INTO stock SELECT c,c,c,c,c,c, 'abc','abc','abc','abc','abc','abc','abc','abc','abc','abc','abc' FROM generate_series(1,3) AS c;
INSERT INTO stock SELECT c, 5000,c,c,c,c, 'abc','abc','abc','abc','abc','abc','abc','abc','abc','abc','abc' FROM generate_series(1,3) AS c; -- mod(2*5000,10000) == 0
INSERT INTO order_line SELECT c, c, c, c, c, '2008-10-17 00:00:00.000000', c, c, c, 'abc' FROM generate_series(0,10) AS c;
INSERT INTO oorder SELECT c, c, c, c, c, 1, 1, '2008-10-17 00:00:00.000000' FROM generate_series(0,10) AS c;
INSERT INTO customer SELECT c, c, c, 0, 'XX', 'John', 'Doe', 1000, 0, 0, c, c, 'Name', 'Street', 'Some City', 'CA', '12345', '+1 000 0000000', '2007-01-02 00:00:00.000000', 'NA', 'nothing special' FROM generate_series(0,10) AS c;
INSERT INTO item SELECT c, 'Keyboard', 50, 'co b', c FROM generate_series(0,10) AS c; --co% and %b filters all around
INSERT INTO region VALUES
(1, 'Not Europe', 'Big'),
(2, 'Europe', 'Big');
INSERT INTO nation VALUES
(1, 'United States', 1, 'Also Big'),
(4, 'The Netherlands', 2, 'Flat'),
(9, 'Germany', 2, 'Germany must be in here for Q7'),
(67, 'Cambodia', 2, 'I don''t understand how we got from California to Cambodia but I will take it, it also is not in Europe, but we need it to be for Q8');

View File

@ -0,0 +1,23 @@
CREATE SCHEMA "te;'st";
SET search_path to "te;'st", public;
CREATE TABLE dist(a int, b int);
SELECT create_distributed_table('dist', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE dist2(a int, b int);
SELECT create_distributed_table('dist2', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE table ref(a int, b int);
SELECT create_reference_table('ref');
create_reference_table
---------------------------------------------------------------------
(1 row)

View File

@ -0,0 +1,521 @@
SET search_path TO "distributed planning";
-- Confirm the basics work
INSERT INTO test VALUES (1, 2), (3, 4), (5, 6), (2, 7), (4, 5);
INSERT INTO test VALUES (6, 7);
SELECT * FROM test WHERE x = 1 ORDER BY y, x;
x | y
---------------------------------------------------------------------
1 | 2
(1 row)
SELECT t1.x, t2.y FROM test t1 JOIN test t2 USING(x) WHERE t1.x = 1 AND t2.x = 1 ORDER BY t2.y, t1.x;
x | y
---------------------------------------------------------------------
1 | 2
(1 row)
SELECT * FROM test WHERE x = 1 OR x = 2 ORDER BY y, x;
x | y
---------------------------------------------------------------------
1 | 2
2 | 7
(2 rows)
SELECT count(*) FROM test;
count
---------------------------------------------------------------------
6
(1 row)
SELECT * FROM test ORDER BY x;
x | y
---------------------------------------------------------------------
1 | 2
2 | 7
3 | 4
4 | 5
5 | 6
6 | 7
(6 rows)
WITH cte_1 AS (UPDATE test SET y = y - 1 RETURNING *) SELECT * FROM cte_1 ORDER BY 1,2;
x | y
---------------------------------------------------------------------
1 | 1
2 | 6
3 | 3
4 | 4
5 | 5
6 | 6
(6 rows)
-- observe that there is a conflict and the following query does nothing
INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT DO NOTHING RETURNING *;
part_key | other_col | third_col
---------------------------------------------------------------------
(0 rows)
-- same as the above with different syntax
INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT (part_key) DO NOTHING RETURNING *;
part_key | other_col | third_col
---------------------------------------------------------------------
(0 rows)
-- again the same query with another syntax
INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO NOTHING RETURNING *;
part_key | other_col | third_col
---------------------------------------------------------------------
(0 rows)
BEGIN;
-- force local execution if possible
SELECT count(*) FROM upsert_test WHERE part_key = 1;
count
---------------------------------------------------------------------
1
(1 row)
-- multi-shard pushdown query that goes through local execution
INSERT INTO upsert_test (part_key, other_col) SELECT part_key, other_col FROM upsert_test ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO NOTHING RETURNING *;
part_key | other_col | third_col
---------------------------------------------------------------------
(0 rows)
INSERT INTO upsert_test (part_key, other_col) SELECT part_key, other_col FROM upsert_test ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO UPDATE SET other_col=EXCLUDED.other_col + 1 RETURNING *;
part_key | other_col | third_col
---------------------------------------------------------------------
1 | 2 |
2 | 3 |
(2 rows)
-- multi-shard pull-to-coordinator query that goes through local execution
INSERT INTO upsert_test (part_key, other_col) SELECT part_key, other_col FROM upsert_test LIMIT 100 ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO NOTHING RETURNING *;
part_key | other_col | third_col
---------------------------------------------------------------------
(0 rows)
INSERT INTO upsert_test (part_key, other_col) SELECT part_key, other_col FROM upsert_test LIMIT 100 ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO UPDATE SET other_col=EXCLUDED.other_col + 1 RETURNING *;
part_key | other_col | third_col
---------------------------------------------------------------------
1 | 3 |
2 | 4 |
(2 rows)
COMMIT;
BEGIN;
INSERT INTO test SELECT i,i from generate_series(0,1000)i;
-- only pulls 1 row, should not hit the limit
WITH cte_1 AS (SELECT * FROM test LIMIT 1) SELECT count(*) FROM cte_1;
count
---------------------------------------------------------------------
1
(1 row)
-- cte_1 only pulls 1 row, but cte_2 all rows
WITH cte_1 AS (SELECT * FROM test LIMIT 1),
cte_2 AS (SELECT * FROM test OFFSET 0)
SELECT count(*) FROM cte_1, cte_2;
count
---------------------------------------------------------------------
1007
(1 row)
ROLLBACK;
-- single shard and multi-shard delete
-- inside a transaction block
BEGIN;
DELETE FROM test WHERE y = 5;
INSERT INTO test VALUES (4, 5);
DELETE FROM test WHERE x = 1;
INSERT INTO test VALUES (1, 2);
COMMIT;
-- basic view queries
CREATE VIEW simple_view AS
SELECT count(*) as cnt FROM test t1 JOIN test t2 USING (x);
SELECT * FROM simple_view;
cnt
---------------------------------------------------------------------
8
(1 row)
SELECT * FROM simple_view, test WHERE test.x = simple_view.cnt;
cnt | x | y
---------------------------------------------------------------------
(0 rows)
BEGIN;
COPY test(x) FROM STDIN;
SELECT count(*) FROM test;
count
---------------------------------------------------------------------
16
(1 row)
COPY (SELECT count(DISTINCT x) FROM test) TO STDOUT;
10
INSERT INTO test SELECT i,i FROM generate_series(0,100)i;
ROLLBACK;
-- prepared statements with custom types
PREPARE single_node_prepare_p1(int, int, new_type) AS
INSERT INTO test_2 VALUES ($1, $2, $3);
EXECUTE single_node_prepare_p1(1, 1, (95, 'citus9.5')::new_type);
EXECUTE single_node_prepare_p1(2 ,2, (94, 'citus9.4')::new_type);
EXECUTE single_node_prepare_p1(3 ,2, (93, 'citus9.3')::new_type);
EXECUTE single_node_prepare_p1(4 ,2, (92, 'citus9.2')::new_type);
EXECUTE single_node_prepare_p1(5 ,2, (91, 'citus9.1')::new_type);
EXECUTE single_node_prepare_p1(6 ,2, (90, 'citus9.0')::new_type);
EXECUTE single_node_prepare_p1(6 ,2, (90, 'citus9.0')::new_type);
PREPARE use_local_query_cache(int) AS SELECT count(*) FROM test_2 WHERE x = $1;
EXECUTE use_local_query_cache(1);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE use_local_query_cache(1);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE use_local_query_cache(1);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE use_local_query_cache(1);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE use_local_query_cache(1);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE use_local_query_cache(1);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE use_local_query_cache(1);
count
---------------------------------------------------------------------
1
(1 row)
BEGIN;
INSERT INTO test_2 VALUES (7 ,2, (83, 'citus8.3')::new_type);
SAVEPOINT s1;
INSERT INTO test_2 VALUES (9 ,1, (82, 'citus8.2')::new_type);
SAVEPOINT s2;
ROLLBACK TO SAVEPOINT s1;
SELECT * FROM test_2 WHERE z = (83, 'citus8.3')::new_type OR z = (82, 'citus8.2')::new_type;
x | y | z
---------------------------------------------------------------------
7 | 2 | (83,citus8.3)
(1 row)
RELEASE SAVEPOINT s1;
COMMIT;
SELECT * FROM test_2 WHERE z = (83, 'citus8.3')::new_type OR z = (82, 'citus8.2')::new_type;
x | y | z
---------------------------------------------------------------------
7 | 2 | (83,citus8.3)
(1 row)
WITH cte_1 AS (SELECT * FROM test_2) SELECT * FROM cte_1 ORDER BY 1,2;
x | y | z
---------------------------------------------------------------------
1 | 1 | (95,citus9.5)
2 | 2 | (94,citus9.4)
3 | 2 | (93,citus9.3)
4 | 2 | (92,citus9.2)
5 | 2 | (91,citus9.1)
6 | 2 | (90,citus9.0)
6 | 2 | (90,citus9.0)
7 | 2 | (83,citus8.3)
(8 rows)
-- final query is router query
WITH cte_1 AS (SELECT * FROM test_2) SELECT * FROM cte_1, test_2 WHERE test_2.x = cte_1.x AND test_2.x = 7 ORDER BY 1,2;
x | y | z | x | y | z
---------------------------------------------------------------------
7 | 2 | (83,citus8.3) | 7 | 2 | (83,citus8.3)
(1 row)
-- final query is a distributed query
WITH cte_1 AS (SELECT * FROM test_2) SELECT * FROM cte_1, test_2 WHERE test_2.x = cte_1.x AND test_2.y != 2 ORDER BY 1,2;
x | y | z | x | y | z
---------------------------------------------------------------------
1 | 1 | (95,citus9.5) | 1 | 1 | (95,citus9.5)
(1 row)
SELECT count(DISTINCT x) FROM test;
count
---------------------------------------------------------------------
5
(1 row)
SELECT count(DISTINCT y) FROM test;
count
---------------------------------------------------------------------
5
(1 row)
-- query pushdown should work
SELECT
*
FROM
(SELECT x, count(*) FROM test_2 GROUP BY x) as foo,
(SELECT x, count(*) FROM test_2 GROUP BY x) as bar
WHERE
foo.x = bar.x
ORDER BY 1 DESC, 2 DESC, 3 DESC, 4 DESC
LIMIT 1;
x | count | x | count
---------------------------------------------------------------------
7 | 1 | 7 | 1
(1 row)
-- Check repartion joins are supported
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x, t2.x, t1.y, t2.y;
x | y | x | y
---------------------------------------------------------------------
2 | 6 | 1 | 2
3 | 3 | 3 | 3
4 | 4 | 4 | 4
4 | 5 | 4 | 4
6 | 6 | 2 | 6
6 | 6 | 6 | 6
(6 rows)
-- INSERT SELECT router
BEGIN;
INSERT INTO test(x, y) SELECT x, y FROM test WHERE x = 1;
SELECT count(*) from test;
count
---------------------------------------------------------------------
7
(1 row)
ROLLBACK;
-- INSERT SELECT pushdown
BEGIN;
INSERT INTO test(x, y) SELECT x, y FROM test;
SELECT count(*) from test;
count
---------------------------------------------------------------------
12
(1 row)
ROLLBACK;
-- INSERT SELECT analytical query
BEGIN;
INSERT INTO test(x, y) SELECT count(x), max(y) FROM test;
SELECT count(*) from test;
count
---------------------------------------------------------------------
7
(1 row)
ROLLBACK;
-- INSERT SELECT repartition
BEGIN;
INSERT INTO test(x, y) SELECT y, x FROM test;
SELECT count(*) from test;
count
---------------------------------------------------------------------
12
(1 row)
ROLLBACK;
-- INSERT SELECT from reference table into distributed
BEGIN;
INSERT INTO test(x, y) SELECT a, b FROM ref;
SELECT count(*) from test;
count
---------------------------------------------------------------------
6
(1 row)
ROLLBACK;
-- INSERT SELECT from local table into distributed
BEGIN;
INSERT INTO test(x, y) SELECT c, d FROM local;
SELECT count(*) from test;
count
---------------------------------------------------------------------
6
(1 row)
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO ref(a, b) SELECT x, y FROM test;
SELECT count(*) from ref;
count
---------------------------------------------------------------------
6
(1 row)
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO ref(a, b) SELECT c, d FROM local;
SELECT count(*) from ref;
count
---------------------------------------------------------------------
0
(1 row)
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO local(c, d) SELECT x, y FROM test;
SELECT count(*) from local;
count
---------------------------------------------------------------------
6
(1 row)
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO local(c, d) SELECT a, b FROM ref;
SELECT count(*) from local;
count
---------------------------------------------------------------------
0
(1 row)
ROLLBACK;
-- Confirm that dummy placements work
SELECT count(*) FROM test WHERE false;
count
---------------------------------------------------------------------
0
(1 row)
SELECT count(*) FROM test WHERE false GROUP BY GROUPING SETS (x,y);
count
---------------------------------------------------------------------
(0 rows)
SELECT count(*) FROM test;
count
---------------------------------------------------------------------
6
(1 row)
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO ref(a, b) SELECT x, y FROM test;
SELECT count(*) from ref;
count
---------------------------------------------------------------------
6
(1 row)
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO ref(a, b) SELECT c, d FROM local;
SELECT count(*) from ref;
count
---------------------------------------------------------------------
0
(1 row)
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO local(c, d) SELECT x, y FROM test;
SELECT count(*) from local;
count
---------------------------------------------------------------------
6
(1 row)
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO local(c, d) SELECT a, b FROM ref;
SELECT count(*) from local;
count
---------------------------------------------------------------------
0
(1 row)
ROLLBACK;
-- query fails on the shards should be handled
-- nicely
\set VERBOSITY terse
SELECT x/0 FROM test;
ERROR: division by zero
SELECT count(DISTINCT row(key, value)) FROM non_binary_copy_test;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*), event FROM date_part_table GROUP BY event ORDER BY count(*) DESC, event DESC LIMIT 5;
count | event
---------------------------------------------------------------------
1 | 100
1 | 99
1 | 98
1 | 97
1 | 96
(5 rows)
SELECT count(*), event FROM date_part_table WHERE user_id = 1 GROUP BY event ORDER BY count(*) DESC, event DESC LIMIT 5;
count | event
---------------------------------------------------------------------
1 | 81
1 | 61
1 | 41
1 | 21
1 | 1
(5 rows)
SELECT count(*), t1.event FROM date_part_table t1 JOIN date_part_table USING (user_id) WHERE t1.user_id = 1 GROUP BY t1.event ORDER BY count(*) DESC, t1.event DESC LIMIT 5;
count | event
---------------------------------------------------------------------
5 | 81
5 | 61
5 | 41
5 | 21
5 | 1
(5 rows)
SELECT count(*), event FROM date_part_table WHERE event_time > '2020-01-05' GROUP BY event ORDER BY count(*) DESC, event DESC LIMIT 5;
count | event
---------------------------------------------------------------------
1 | 100
1 | 99
1 | 98
1 | 97
1 | 96
(5 rows)
SELECT count(*), event FROM date_part_table WHERE user_id = 12 AND event_time = '2020-01-12 12:00:00' GROUP BY event ORDER BY count(*) DESC, event DESC LIMIT 5;
count | event
---------------------------------------------------------------------
1 | 92
(1 row)
SELECT count(*), t1.event FROM date_part_table t1 JOIN date_part_table t2 USING (user_id) WHERE t1.user_id = 1 AND t2.event_time > '2020-01-03' GROUP BY t1.event ORDER BY count(*) DESC, t1.event DESC LIMIT 5;
count | event
---------------------------------------------------------------------
4 | 81
4 | 61
4 | 41
4 | 21
4 | 1
(5 rows)

View File

@ -0,0 +1,85 @@
CREATE SCHEMA "distributed planning";
SET search_path TO "distributed planning";
CREATE TABLE
date_part_table (event_time timestamp, event int, user_id int)
partition by range (event_time);
SELECT create_distributed_table('date_part_table', 'user_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- an unnamed index
CREATE INDEX ON date_part_table(user_id, event_time);
--create named and long index with parameters
CREATE INDEX event_idx ON date_part_table(event, event_time, user_id, user_id, event_time);
SELECT create_time_partitions(table_name:='date_part_table',
partition_interval:= '1 week',
end_at:= '2020-01-20',
start_from:='2020-01-01');
create_time_partitions
---------------------------------------------------------------------
t
(1 row)
INSERT INTO date_part_table
SELECT '2020-01-01'::timestamp + '3 hours'::interval * i, i, i % 20 FROM generate_series(0,100)i;
CREATE TABLE test(x int, y int);
SELECT create_distributed_table('test','x');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TYPE new_type AS (n int, m text);
CREATE TABLE test_2(x int, y int, z new_type);
SELECT create_distributed_table('test_2','x');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE ref(a int, b int);
SELECT create_reference_table('ref');
create_reference_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE local(c int, d int);
select citus_add_local_table_to_metadata('local');
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
CREATE TABLE non_binary_copy_test (key int PRIMARY KEY, value new_type);
SELECT create_distributed_table('non_binary_copy_test', 'key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO non_binary_copy_test SELECT i, (i, 'citus9.5')::new_type FROM generate_series(0,1000)i;
-- Test upsert with constraint
CREATE TABLE upsert_test
(
part_key int UNIQUE,
other_col int,
third_col int
);
-- distribute the table
SELECT create_distributed_table('upsert_test', 'part_key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- do a regular insert
INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1), (2, 2) RETURNING *;
part_key | other_col | third_col
---------------------------------------------------------------------
1 | 1 |
2 | 2 |
(2 rows)

View File

@ -0,0 +1,303 @@
SET search_path TO local_shard_execution_dropped_column;
prepare p1(int) as insert into t1(a,c) VALUES (5,$1) ON CONFLICT (c) DO NOTHING;
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
prepare p2(int) as SELECT count(*) FROM t1 WHERE c = $1 GROUP BY c;
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
execute p2(8);
count
---------------------------------------------------------------------
1
(1 row)
prepare p3(int) as INSERT INTO t1(a,c) VALUES (5, $1), (6, $1), (7, $1),(5, $1), (6, $1), (7, $1) ON CONFLICT DO NOTHING;
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
prepare p4(int) as UPDATE t1 SET a = a + 1 WHERE c = $1;
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
-- https://github.com/citusdata/citus/issues/5355
-- prepare p5(int) as INSERT INTO t1(a,c) VALUES (15, $1) ON CONFLICT (c) DO UPDATE SET a=EXCLUDED.a + 10 RETURNING *;
-- execute p5(18);
-- execute p5(19);
-- execute p5(20);
-- execute p5(21);
-- execute p5(22);
-- execute p5(23);
-- execute p5(24);
-- execute p5(25);
-- execute p5(26);
-- execute p5(27);
-- execute p5(28);
-- execute p5(29);
-- show that all the tables prune to the same shard for the same distribution key
WITH
sensors_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors', 3)),
sensors_2000_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2000', 3)),
sensors_2001_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2001', 3)),
sensors_2002_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2002', 3)),
sensors_2003_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2003', 3)),
sensors_2004_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2004', 3)),
all_shardids AS (SELECT * FROM sensors_shardid UNION SELECT * FROM sensors_2000_shardid UNION
SELECT * FROM sensors_2001_shardid UNION SELECT * FROM sensors_2002_shardid
UNION SELECT * FROM sensors_2003_shardid UNION SELECT * FROM sensors_2004_shardid)
-- it is zero for PG only tests, and 1 for Citus
SELECT count(DISTINCT row(shardminvalue, shardmaxvalue)) <= 1 FROM pg_dist_shard WHERE shardid IN (SELECT * FROM all_shardids);
?column?
---------------------------------------------------------------------
t
(1 row)
INSERT INTO sensors VALUES (3, '2000-02-02', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2000-01-01', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2001-01-01', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2002-01-01', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2003-01-01', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2004-01-01', row_to_json(row(1)));
SELECT count(*) FROM sensors WHERE measureid = 3 AND eventdatetime = '2000-02-02';
count
---------------------------------------------------------------------
1
(1 row)
SELECT count(*) FROM sensors_2000 WHERE measureid = 3;
count
---------------------------------------------------------------------
2
(1 row)
SELECT count(*) FROM sensors_2001 WHERE measureid = 3;
count
---------------------------------------------------------------------
1
(1 row)
SELECT count(*) FROM sensors_2002 WHERE measureid = 3;
count
---------------------------------------------------------------------
1
(1 row)
SELECT count(*) FROM sensors_2003 WHERE measureid = 3;
count
---------------------------------------------------------------------
1
(1 row)
-- multi-shard queries
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors;
count
---------------------------------------------------------------------
6
(1 row)
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2000;
count
---------------------------------------------------------------------
2
(1 row)
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2001;
count
---------------------------------------------------------------------
1
(1 row)
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2002;
count
---------------------------------------------------------------------
1
(1 row)
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2003;
count
---------------------------------------------------------------------
1
(1 row)
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2004;
count
---------------------------------------------------------------------
1
(1 row)
-- execute 7 times to make sure it is re-cached
-- prepared statements should work fine even after columns are dropped
PREPARE drop_col_prepare_insert(int, date, jsonb) AS INSERT INTO sensors (measureid, eventdatetime, measure_data) VALUES ($1, $2, $3);
PREPARE drop_col_prepare_select(int, date) AS SELECT count(*) FROM sensors WHERE measureid = $1 AND eventdatetime = $2;
PREPARE drop_col_prepare_mshard_select(date) AS SELECT count(*) FROM sensors WHERE eventdatetime = $1;
EXECUTE drop_col_prepare_insert(3, '2000-10-01', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(3, '2001-10-01', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(3, '2002-10-01', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(3, '2003-10-01', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(3, '2003-10-02', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(4, '2003-10-03', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(5, '2003-10-04', row_to_json(row(1)));
EXECUTE drop_col_prepare_select(3, '2000-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_select(3, '2001-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_select(3, '2002-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_select(3, '2003-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_select(3, '2003-10-02');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_select(4, '2003-10-03');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_select(5, '2003-10-04');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_mshard_select('2000-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_mshard_select('2000-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_mshard_select('2001-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_mshard_select('2002-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_mshard_select('2002-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_mshard_select('2003-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_mshard_select('2003-10-01');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE drop_col_prepare_mshard_select('2004-01-01');
count
---------------------------------------------------------------------
1
(1 row)

View File

@ -0,0 +1,69 @@
CREATE SCHEMA local_shard_execution_dropped_column;
SET search_path TO local_shard_execution_dropped_column;
CREATE TABLE t1 (a int, b int, c int UNIQUE, d int, e int);
ALTER TABLE t1 DROP COLUMN e;
SELECT create_distributed_table('t1', 'c');
create_distributed_table
---------------------------------------------------------------------
(1 row)
ALTER TABLE t1 DROP COLUMN b;
ALTER TABLE t1 DROP COLUMN d;
-- create a partitioned table with some columns that
-- are going to be dropped within the tests
CREATE TABLE sensors(
col_to_drop_0 text,
col_to_drop_1 text,
col_to_drop_2 date,
col_to_drop_3 inet,
col_to_drop_4 date,
measureid integer,
eventdatetime date,
measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data))
PARTITION BY RANGE(eventdatetime);
-- drop column even before attaching any partitions
ALTER TABLE sensors DROP COLUMN col_to_drop_1;
-- now attach the first partition and create the distributed table
CREATE TABLE sensors_2000 PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2001-01-01');
SELECT create_distributed_table('sensors', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- drop another column before attaching another partition
-- with .. PARTITION OF .. syntax
ALTER TABLE sensors DROP COLUMN col_to_drop_0;
CREATE TABLE sensors_2001 PARTITION OF sensors FOR VALUES FROM ('2001-01-01') TO ('2002-01-01');
-- drop another column before attaching another partition
-- with ALTER TABLE .. ATTACH PARTITION
ALTER TABLE sensors DROP COLUMN col_to_drop_2;
CREATE TABLE sensors_2002(
col_to_drop_4 date, col_to_drop_3 inet, measureid integer, eventdatetime date, measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data));
ALTER TABLE sensors ATTACH PARTITION sensors_2002 FOR VALUES FROM ('2002-01-01') TO ('2003-01-01');
-- drop another column before attaching another partition
-- that is already distributed
ALTER TABLE sensors DROP COLUMN col_to_drop_3;
CREATE TABLE sensors_2003(
col_to_drop_4 date, measureid integer, eventdatetime date, measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data));
SELECT create_distributed_table('sensors_2003', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
ALTER TABLE sensors ATTACH PARTITION sensors_2003 FOR VALUES FROM ('2003-01-01') TO ('2004-01-01');
CREATE TABLE sensors_2004(
col_to_drop_4 date, measureid integer NOT NULL, eventdatetime date NOT NULL, measure_data jsonb NOT NULL);
ALTER TABLE sensors ATTACH PARTITION sensors_2004 FOR VALUES FROM ('2004-01-01') TO ('2005-01-01');
ALTER TABLE sensors DROP COLUMN col_to_drop_4;
SELECT alter_table_set_access_method('sensors_2004', 'columnar');
alter_table_set_access_method
---------------------------------------------------------------------
(1 row)

View File

@ -0,0 +1,51 @@
CREATE SCHEMA "intermediate result pruning";
SET search_path TO "intermediate result pruning";
CREATE TABLE table_1 (key int, value text);
SELECT create_distributed_table('table_1', 'key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE table_2 (key int, value text);
SELECT create_distributed_table('table_2', 'key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE table_3 (key int, value text);
SELECT create_distributed_table('table_3', 'key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE ref_table (key int, value text);
SELECT create_reference_table('ref_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
-- load some data
INSERT INTO table_1 VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4');
INSERT INTO table_2 VALUES (3, '3'), (4, '4'), (5, '5'), (6, '6');
INSERT INTO table_3 VALUES (3, '3'), (4, '4'), (5, '5'), (6, '6');
INSERT INTO ref_table VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6');
CREATE TABLE accounts (id text PRIMARY KEY);
CREATE TABLE stats (account_id text PRIMARY KEY, spent int);
SELECT create_distributed_table('accounts', 'id', colocate_with => 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('stats', 'account_id', colocate_with => 'accounts');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO accounts (id) VALUES ('foo');
INSERT INTO stats (account_id, spent) VALUES ('foo', 100);

View File

@ -0,0 +1,509 @@
SET search_path TO "intermediate result pruning";
-- a very basic case, where the intermediate result
-- should go to both workers
WITH some_values_1 AS MATERIALIZED
(SELECT key FROM table_1 WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
some_values_1 JOIN table_2 USING (key);
count
---------------------------------------------------------------------
2
(1 row)
-- a very basic case, where the intermediate result
-- should only go to one worker because the final query is a router
-- we use random() to prevent postgres inline the CTE(s)
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
some_values_1 JOIN table_2 USING (key) WHERE table_2.key = 3;
count
---------------------------------------------------------------------
1
(1 row)
-- a similar query, but with a reference table now
-- given that reference tables are replicated to all nodes
-- we have to broadcast to all nodes
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
some_values_1 JOIN ref_table USING (key);
count
---------------------------------------------------------------------
2
(1 row)
-- a similar query as above, but this time use the CTE inside
-- another CTE
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1)
SELECT
count(*)
FROM
some_values_2 JOIN table_2 USING (key) WHERE table_2.key = 3;
count
---------------------------------------------------------------------
1
(1 row)
-- the second CTE does a join with a distributed table
-- and the final query is a router query
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key))
SELECT
count(*)
FROM
some_values_2 JOIN table_2 USING (key) WHERE table_2.key = 3;
count
---------------------------------------------------------------------
1
(1 row)
-- the first CTE is used both within second CTE and the final query
-- the second CTE does a join with a distributed table
-- and the final query is a router query
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key))
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key = 3;
count
---------------------------------------------------------------------
1
(1 row)
-- the first CTE is used both within second CTE and the final query
-- the second CTE does a join with a distributed table but a router query on a worker
-- and the final query is another router query on another worker
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE table_2.key = 4)
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key = 4;
count
---------------------------------------------------------------------
1
(1 row)
-- the first CTE is used both within second CTE and the final query
-- the second CTE does a join with a distributed table but a router query on a worker
-- and the final query is a router query on the same worker, so the first result is only
-- broadcasted to a single node
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE table_2.key = 3)
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key = 3;
count
---------------------------------------------------------------------
1
(1 row)
-- the same query with the above, but the final query is hitting all shards
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key))
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key != 3;
count
---------------------------------------------------------------------
1
(1 row)
-- even if we add a filter on the first query and make it a router query,
-- the first intermediate result still hits all workers because of the final
-- join is hitting all workers
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE table_2.key = 3)
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key != 4;
count
---------------------------------------------------------------------
1
(1 row)
-- the reference table is joined with a distributed table and an intermediate
-- result, but the distributed table hits all shards, so the intermediate
-- result is sent to all nodes
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM ref_table WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
(some_values_1 JOIN ref_table USING (key)) JOIN table_2 USING (key);
count
---------------------------------------------------------------------
2
(1 row)
-- similar query as above, but this time the whole query is a router
-- query, so no intermediate results
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM ref_table WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
(some_values_1 JOIN ref_table USING (key)) JOIN table_2 USING (key) WHERE table_2.key = 4;
count
---------------------------------------------------------------------
1
(1 row)
-- now, the second CTE has a single shard join with a distributed table
-- so the first CTE should only be broadcasted to that node
-- since the final query doesn't have a join, it should simply be broadcasted
-- to one node
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE key = 1)
SELECT
count(*)
FROM
some_values_2;
count
---------------------------------------------------------------------
0
(1 row)
-- the same query inlined inside a CTE, and the final query has a
-- join with a distributed table
WITH top_cte as MATERIALIZED (
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE key = 4)
SELECT
DISTINCT key
FROM
some_values_2
)
SELECT
count(*)
FROM
top_cte JOIN table_2 USING (key);
count
---------------------------------------------------------------------
1
(1 row)
-- very much the same query, but this time the top query is also a router query
-- on a single worker, so all intermediate results only hit a single node
WITH top_cte as MATERIALIZED (
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE key = 1)
SELECT
DISTINCT key
FROM
some_values_2
)
SELECT
count(*)
FROM
top_cte JOIN table_2 USING (key) WHERE table_2.key = 2;
count
---------------------------------------------------------------------
0
(1 row)
-- some_values_1 is first used by a single shard-query, and than with a multi-shard
-- CTE, finally a cartesian product join
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('6', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE key = 4),
some_values_3 AS MATERIALIZED
(SELECT key FROM (some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key))
SELECT * FROM some_values_3 JOIN ref_table ON (true) ORDER BY 1,2,3;
key | key | value
---------------------------------------------------------------------
4 | 1 | 1
4 | 2 | 2
4 | 3 | 3
4 | 4 | 4
4 | 5 | 5
4 | 6 | 6
(6 rows)
-- join on intermediate results, so should only
-- go to a single node
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM table_2 WHERE value IN ('3', '4'))
SELECT count(*) FROM some_values_2 JOIN some_values_1 USING (key);
count
---------------------------------------------------------------------
2
(1 row)
-- same query with WHERE false make sure that we're not broken
-- for such edge cases
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM table_2 WHERE value IN ('3', '4'))
SELECT count(*) FROM some_values_2 JOIN some_values_1 USING (key) WHERE false;
count
---------------------------------------------------------------------
0
(1 row)
-- do not use some_values_2 at all, so only 2 intermediate results are
-- broadcasted
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1),
some_values_3 AS MATERIALIZED
(SELECT key, random() FROM some_values_1)
SELECT
count(*)
FROM
some_values_3;
count
---------------------------------------------------------------------
2
(1 row)
-- lets have some deeper intermediate results
-- the inner most two results and the final query (which contains only intermediate results)
-- hitting single worker, others hitting all workers
-- (see below query where all intermediate results hit a single node)
SELECT count(*) FROM
(
SELECT avg(min::int) FROM
(
SELECT min(table_1.value) FROM
(
SELECT avg(value::int) as avg_ev_type FROM
(
SELECT max(value) as mx_val_1 FROM
(
SELECT avg(value::int) as avg FROM
(
SELECT cnt FROM
(
SELECT count(*) as cnt, value
FROM table_1
WHERE key = 1
GROUP BY value
) as level_1, table_1
WHERE table_1.key = level_1.cnt AND key = 3
) as level_2, table_2
WHERE table_2.key = level_2.cnt AND key = 5
GROUP BY level_2.cnt
) as level_3, table_1
WHERE value::numeric = level_3.avg AND key = 6
GROUP BY level_3.avg
) as level_4, table_2
WHERE level_4.mx_val_1::int = table_2.key
GROUP BY level_4.mx_val_1
) as level_5, table_1
WHERE level_5.avg_ev_type = table_1.key AND key > 111
GROUP BY level_5.avg_ev_type
) as level_6, table_1 WHERE table_1.key::int = level_6.min::int
GROUP BY table_1.value
) as bar;
count
---------------------------------------------------------------------
0
(1 row)
-- the same query where all intermediate results hits one
-- worker because each and every query is a router query -- but on different nodes
SELECT count(*) FROM
(
SELECT avg(min::int) FROM
(
SELECT min(table_1.value) FROM
(
SELECT avg(value::int) as avg_ev_type FROM
(
SELECT max(value) as mx_val_1 FROM
(
SELECT avg(value::int) as avg FROM
(
SELECT cnt FROM
(
SELECT count(*) as cnt, value
FROM table_1
WHERE key = 1
GROUP BY value
) as level_1, table_1
WHERE table_1.key = level_1.cnt AND key = 3
) as level_2, table_2
WHERE table_2.key = level_2.cnt AND key = 5
GROUP BY level_2.cnt
) as level_3, table_1
WHERE value::numeric = level_3.avg AND key = 6
GROUP BY level_3.avg
) as level_4, table_2
WHERE level_4.mx_val_1::int = table_2.key AND table_2.key = 1
GROUP BY level_4.mx_val_1
) as level_5, table_1
WHERE level_5.avg_ev_type = table_1.key AND key = 111
GROUP BY level_5.avg_ev_type
) as level_6, table_1
WHERE table_1.key::int = level_6.min::int AND table_1.key = 4
GROUP BY table_1.value
) as bar;
count
---------------------------------------------------------------------
0
(1 row)
-- sanity checks for set operations
-- the intermediate results should just hit a single worker
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2);
key
---------------------------------------------------------------------
(0 rows)
-- the intermediate results should just hit a single worker
WITH cte_1 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2)
),
cte_2 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 3)
INTERSECT
(SELECT key FROM table_1 WHERE key = 4)
)
SELECT * FROM cte_1
UNION
SELECT * FROM cte_2;
key
---------------------------------------------------------------------
(0 rows)
-- one final test with SET operations, where
-- we join the results with distributed tables
-- so cte_1 should hit all workers, but still the
-- others should hit single worker each
WITH cte_1 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2)
),
cte_2 AS MATERIALIZED
(
SELECT count(*) FROM table_1 JOIN cte_1 USING (key)
)
SELECT * FROM cte_2;
count
---------------------------------------------------------------------
0
(1 row)
-- sanity checks for non-colocated subquery joins
-- the recursively planned subquery (bar) should hit all
-- nodes
SELECT
count(*)
FROM
(SELECT key, random() FROM table_1) as foo,
(SELECT key, random() FROM table_2) as bar
WHERE
foo.key != bar.key;
count
---------------------------------------------------------------------
14
(1 row)
-- the recursively planned subquery (bar) should hit one
-- node because foo goes to a single node
SELECT
count(*)
FROM
(SELECT key, random() FROM table_1 WHERE key = 1) as foo,
(SELECT key, random() FROM table_2) as bar
WHERE
foo.key != bar.key;
count
---------------------------------------------------------------------
4
(1 row)
SELECT *
FROM
(
WITH accounts_cte AS MATERIALIZED (
SELECT id AS account_id
FROM accounts
),
joined_stats_cte_1 AS MATERIALIZED (
SELECT spent, account_id
FROM stats
INNER JOIN accounts_cte USING (account_id)
),
joined_stats_cte_2 AS MATERIALIZED (
SELECT spent, account_id
FROM joined_stats_cte_1
INNER JOIN accounts_cte USING (account_id)
)
SELECT SUM(spent) OVER (PARTITION BY coalesce(account_id, NULL))
FROM accounts_cte
INNER JOIN joined_stats_cte_2 USING (account_id)
) inner_query;
sum
---------------------------------------------------------------------
100
(1 row)
-- Testing a having clause that could have been a where clause between a distributed table
-- and a reference table. This query was the cause for intermediate results not being
-- available during the replace of the planner for the master query with the standard
-- planner.
-- Since the having clause could have been a where clause the having clause on the grouping
-- on the coordinator is replaced with a Result node containing a One-time filter if the
-- having qual (one-time filter works because the query doesn't change with the tuples
-- returned from below).
SELECT count(*),
spent
FROM stats
GROUP BY 2
HAVING (
SELECT count(*)
FROM accounts
) > 0;
count | spent
---------------------------------------------------------------------
1 | 100
(1 row)

View File

@ -0,0 +1,143 @@
SET search_path TO "intermediate result pruning";
-- sanity checks for modification queries
-- select_data goes to a single node, because it is used in another subquery
-- raw_data is also the final router query, so hits a single shard
-- however, the subquery in WHERE clause of the DELETE query is broadcasted to all
-- nodes
BEGIN;
WITH select_data AS MATERIALIZED (
SELECT * FROM table_1
),
raw_data AS MATERIALIZED (
DELETE FROM table_2 WHERE key >= (SELECT min(key) FROM select_data WHERE key > 1) RETURNING *
)
SELECT * FROM raw_data ORDER BY 1,2;
key | value
---------------------------------------------------------------------
3 | 3
4 | 4
5 | 5
6 | 6
(4 rows)
ROLLBACK;
-- select_data goes to a single node, because it is used in another subquery
-- raw_data is also the final router query, so hits a single shard
-- however, the subquery in WHERE clause of the DELETE query is broadcasted to all
-- nodes
BEGIN;
WITH select_data AS MATERIALIZED (
SELECT * FROM table_1
),
raw_data AS MATERIALIZED (
DELETE FROM table_2 WHERE value::int >= (SELECT min(key) FROM select_data WHERE key > 1 + random()) RETURNING *
)
SELECT * FROM raw_data ORDER BY 1,2;
key | value
---------------------------------------------------------------------
3 | 3
4 | 4
5 | 5
6 | 6
(4 rows)
ROLLBACK;
-- now, we need only two intermediate results as the subquery in WHERE clause is
-- router plannable
BEGIN;
WITH select_data AS MATERIALIZED (
SELECT * FROM table_1
),
raw_data AS MATERIALIZED (
DELETE FROM table_2 WHERE value::int >= (SELECT min(key) FROM table_1 WHERE key > random()) AND key = 6 RETURNING *
)
SELECT * FROM raw_data ORDER BY 1,2;
key | value
---------------------------------------------------------------------
6 | 6
(1 row)
ROLLBACK;
-- test with INSERT SELECT via coordinator
-- INSERT .. SELECT via coordinator that doesn't have any intermediate results
-- We use offset 1 to make sure the result needs to be pulled to the coordinator, offset 0 would be optimized away
BEGIN;
INSERT INTO table_1
SELECT * FROM table_2 OFFSET 1;
ROLLBACK;
-- INSERT .. SELECT via coordinator which has intermediate result,
-- and can be pruned to a single worker because the final query is on
-- single shard via filter in key
BEGIN;
INSERT INTO table_1
SELECT * FROM table_2 where value IN (SELECT value FROM table_1 WHERE random() > 1) AND key = 1;
ROLLBACK;
-- a similar query, with more complex subquery
BEGIN;
INSERT INTO table_1
SELECT * FROM table_2 where key = 1 AND
value::int IN
(WITH cte_1 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2)
),
cte_2 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 3)
INTERSECT
(SELECT key FROM table_1 WHERE key = 4)
)
SELECT * FROM cte_1
UNION
SELECT * FROM cte_2);
ROLLBACK;
-- same query, cte is on the FROM clause
-- and this time the final query (and top-level intermediate result)
-- hits all the shards because table_2.key != 1
BEGIN;
INSERT INTO table_1
SELECT table_2.* FROM table_2,
(WITH cte_1 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2)
),
cte_2 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 3)
INTERSECT
(SELECT key FROM table_1 WHERE key = 4)
)
SELECT * FROM cte_1
UNION
SELECT * FROM cte_2
) foo
where table_2.key != 1 AND
foo.key = table_2.value::int;
ROLLBACK;
BEGIN;
-- Insert..select is planned differently, make sure we have results everywhere.
-- We put the insert..select in a CTE here to prevent the CTE from being moved
-- into the select, which would follow the regular code path for select.
WITH stats AS MATERIALIZED (
SELECT count(key) m FROM table_3
),
inserts AS MATERIALIZED (
INSERT INTO table_2
SELECT key, count(*)
FROM table_1
WHERE key >= (SELECT m FROM stats)
GROUP BY key
HAVING count(*) < (SELECT m FROM stats)
LIMIT 1
RETURNING *
) SELECT count(*) FROM inserts;
count
---------------------------------------------------------------------
1
(1 row)
ROLLBACK;

View File

@ -0,0 +1,26 @@
-- We override the distribute functions so that we can test postgres configs easily.
CREATE OR REPLACE FUNCTION pg_catalog.create_distributed_table(table_name regclass,
distribution_column text,
distribution_type citus.distribution_type DEFAULT 'hash',
colocate_with text DEFAULT 'default',
shard_count int DEFAULT NULL)
RETURNS void
LANGUAGE plpgsql
AS $function$
BEGIN
END;
$function$;
CREATE OR REPLACE FUNCTION pg_catalog.create_reference_table(table_name regclass)
RETURNS void
LANGUAGE plpgsql
AS $function$
BEGIN
END;
$function$;
CREATE OR REPLACE FUNCTION pg_catalog.citus_add_local_table_to_metadata(table_name regclass, cascade_via_foreign_keys boolean default false)
RETURNS void
LANGUAGE plpgsql
AS $function$
BEGIN
END;
$function$;

View File

@ -0,0 +1,26 @@
SET search_path TO "prepared statements";
PREPARE repartition_prepared(int) AS
SELECT
count(*)
FROM
repartition_prepared_test t1
JOIN
repartition_prepared_test t2
USING (b)
WHERE t1.a = $1;
EXECUTE repartition_prepared (1);
count
---------------------------------------------------------------------
100
(1 row)
BEGIN;
-- CREATE TABLE ... AS EXECUTE prepared_statement tests
CREATE TEMP TABLE repartition_prepared_tmp AS EXECUTE repartition_prepared(1);
SELECT count(*) from repartition_prepared_tmp;
count
---------------------------------------------------------------------
1
(1 row)
ROLLBACK;

View File

@ -0,0 +1,646 @@
SET search_path TO "prepared statements";
-- test parameterized inserts
PREPARE prepared_insert(varchar(20)) AS
INSERT INTO router_executor_table VALUES (1, $1, $2);
EXECUTE prepared_insert('comment-1', '(1, 10)');
EXECUTE prepared_insert('comment-2', '(2, 20)');
EXECUTE prepared_insert('comment-3', '(3, 30)');
EXECUTE prepared_insert('comment-4', '(4, 40)');
EXECUTE prepared_insert('comment-5', '(5, 50)');
EXECUTE prepared_insert('comment-6', '(6, 60)');
EXECUTE prepared_insert('comment-7', '(7, 67)');
-- to make this work, Citus adds the type casting for composite keys
-- during the deparsing
PREPARE prepared_custom_type_select(test_composite_type) AS
SELECT count(*) FROM router_executor_table WHERE id = 1 AND stats = $1;
EXECUTE prepared_custom_type_select('(1,10)');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_custom_type_select('(2,20)');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_custom_type_select('(3,30)');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_custom_type_select('(4,40)');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_custom_type_select('(5,50)');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_custom_type_select('(6,60)');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_custom_type_select('(7,67)');
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_custom_type_select('(7,67)');
count
---------------------------------------------------------------------
1
(1 row)
-- test parameterized selects
PREPARE prepared_select(integer, integer) AS
SELECT count(*) FROM router_executor_table
WHERE id = 1 AND stats = ROW($1, $2)::test_composite_type;
EXECUTE prepared_select(1, 10);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_select(2, 20);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_select(3, 30);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_select(4, 40);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_select(5, 50);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_select(6, 60);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_select(7, 67);
count
---------------------------------------------------------------------
1
(1 row)
EXECUTE prepared_select(7, 67);
count
---------------------------------------------------------------------
1
(1 row)
-- Test that parameterized partition column for an insert is supported
PREPARE prepared_partition_column_insert(bigint) AS
INSERT INTO router_executor_table VALUES ($1, 'arsenous', '(1,10)');
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_partition_column_insert(1);
EXECUTE prepared_partition_column_insert(2);
EXECUTE prepared_partition_column_insert(3);
EXECUTE prepared_partition_column_insert(4);
EXECUTE prepared_partition_column_insert(5);
EXECUTE prepared_partition_column_insert(6);
EXECUTE prepared_partition_column_insert(7);
PREPARE prepared_no_parameter_insert AS
INSERT INTO prepare_table (key) VALUES (0);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
PREPARE prepared_single_parameter_insert(int) AS
INSERT INTO prepare_table (key) VALUES ($1);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_single_parameter_insert(1);
EXECUTE prepared_single_parameter_insert(2);
EXECUTE prepared_single_parameter_insert(3);
EXECUTE prepared_single_parameter_insert(4);
EXECUTE prepared_single_parameter_insert(5);
EXECUTE prepared_single_parameter_insert(6);
EXECUTE prepared_single_parameter_insert(7);
PREPARE prepared_double_parameter_insert(int, int) AS
INSERT INTO prepare_table (key, value) VALUES ($1, $2);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_double_parameter_insert(1, 10);
EXECUTE prepared_double_parameter_insert(2, 20);
EXECUTE prepared_double_parameter_insert(3, 30);
EXECUTE prepared_double_parameter_insert(4, 40);
EXECUTE prepared_double_parameter_insert(5, 50);
EXECUTE prepared_double_parameter_insert(6, 60);
EXECUTE prepared_double_parameter_insert(7, 70);
PREPARE prepared_multi_insert(int, int) AS
INSERT INTO prepare_table (key, value) VALUES ($1, $2), ($1 + 1, $2 + 10);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_multi_insert( 7, 70);
EXECUTE prepared_multi_insert( 9, 90);
EXECUTE prepared_multi_insert(11, 110);
EXECUTE prepared_multi_insert(13, 130);
EXECUTE prepared_multi_insert(15, 150);
EXECUTE prepared_multi_insert(17, 170);
EXECUTE prepared_multi_insert(19, 190);
PREPARE prepared_non_partition_parameter_insert(int) AS
INSERT INTO prepare_table (key, value) VALUES (0, $1);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_non_partition_parameter_insert(10);
EXECUTE prepared_non_partition_parameter_insert(20);
EXECUTE prepared_non_partition_parameter_insert(30);
EXECUTE prepared_non_partition_parameter_insert(40);
EXECUTE prepared_non_partition_parameter_insert(50);
EXECUTE prepared_non_partition_parameter_insert(60);
EXECUTE prepared_non_partition_parameter_insert(70);
-- check inserted values
SELECT count(*) FROM prepare_table;
count
---------------------------------------------------------------------
42
(1 row)
DELETE FROM prepare_table WHERE value >= 70;
-- check router executor select
PREPARE prepared_router_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.key = $1
ORDER BY
key,
value;
EXECUTE prepared_router_partition_column_select(1);
key | value
---------------------------------------------------------------------
1 | 10
1 |
(2 rows)
EXECUTE prepared_router_partition_column_select(2);
key | value
---------------------------------------------------------------------
2 | 20
2 |
(2 rows)
EXECUTE prepared_router_partition_column_select(3);
key | value
---------------------------------------------------------------------
3 | 30
3 |
(2 rows)
EXECUTE prepared_router_partition_column_select(4);
key | value
---------------------------------------------------------------------
4 | 40
4 |
(2 rows)
EXECUTE prepared_router_partition_column_select(5);
key | value
---------------------------------------------------------------------
5 | 50
5 |
(2 rows)
EXECUTE prepared_router_partition_column_select(6);
key | value
---------------------------------------------------------------------
6 | 60
6 |
(2 rows)
EXECUTE prepared_router_partition_column_select(7);
key | value
---------------------------------------------------------------------
7 |
(1 row)
PREPARE prepared_router_non_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.key = 0 AND
prepare_table.value = $1
ORDER BY
key,
value;
EXECUTE prepared_router_non_partition_column_select(10);
key | value
---------------------------------------------------------------------
0 | 10
(1 row)
EXECUTE prepared_router_non_partition_column_select(20);
key | value
---------------------------------------------------------------------
0 | 20
(1 row)
EXECUTE prepared_router_non_partition_column_select(30);
key | value
---------------------------------------------------------------------
0 | 30
(1 row)
EXECUTE prepared_router_non_partition_column_select(40);
key | value
---------------------------------------------------------------------
0 | 40
(1 row)
EXECUTE prepared_router_non_partition_column_select(50);
key | value
---------------------------------------------------------------------
0 | 50
(1 row)
EXECUTE prepared_router_non_partition_column_select(60);
key | value
---------------------------------------------------------------------
0 | 60
(1 row)
EXECUTE prepared_router_non_partition_column_select(67);
key | value
---------------------------------------------------------------------
(0 rows)
-- check real-time executor
PREPARE prepared_real_time_non_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.value = $1
ORDER BY
key,
value;
EXECUTE prepared_real_time_non_partition_column_select(10);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
(2 rows)
EXECUTE prepared_real_time_non_partition_column_select(20);
key | value
---------------------------------------------------------------------
0 | 20
2 | 20
(2 rows)
EXECUTE prepared_real_time_non_partition_column_select(30);
key | value
---------------------------------------------------------------------
0 | 30
3 | 30
(2 rows)
EXECUTE prepared_real_time_non_partition_column_select(40);
key | value
---------------------------------------------------------------------
0 | 40
4 | 40
(2 rows)
EXECUTE prepared_real_time_non_partition_column_select(50);
key | value
---------------------------------------------------------------------
0 | 50
5 | 50
(2 rows)
EXECUTE prepared_real_time_non_partition_column_select(60);
key | value
---------------------------------------------------------------------
0 | 60
6 | 60
(2 rows)
EXECUTE prepared_real_time_non_partition_column_select(70);
key | value
---------------------------------------------------------------------
(0 rows)
PREPARE prepared_real_time_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.key = $1 OR
prepare_table.value = 10
ORDER BY
key,
value;
EXECUTE prepared_real_time_partition_column_select(1);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
1 |
(3 rows)
EXECUTE prepared_real_time_partition_column_select(2);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
2 | 20
2 |
(4 rows)
EXECUTE prepared_real_time_partition_column_select(3);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
3 | 30
3 |
(4 rows)
EXECUTE prepared_real_time_partition_column_select(4);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
4 | 40
4 |
(4 rows)
EXECUTE prepared_real_time_partition_column_select(5);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
5 | 50
5 |
(4 rows)
EXECUTE prepared_real_time_partition_column_select(6);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
6 | 60
6 |
(4 rows)
EXECUTE prepared_real_time_partition_column_select(7);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
7 |
(3 rows)
PREPARE prepared_task_tracker_non_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.value = $1
ORDER BY
key,
value;
EXECUTE prepared_task_tracker_non_partition_column_select(10);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
(2 rows)
EXECUTE prepared_task_tracker_non_partition_column_select(20);
key | value
---------------------------------------------------------------------
0 | 20
2 | 20
(2 rows)
EXECUTE prepared_task_tracker_non_partition_column_select(30);
key | value
---------------------------------------------------------------------
0 | 30
3 | 30
(2 rows)
EXECUTE prepared_task_tracker_non_partition_column_select(40);
key | value
---------------------------------------------------------------------
0 | 40
4 | 40
(2 rows)
EXECUTE prepared_task_tracker_non_partition_column_select(50);
key | value
---------------------------------------------------------------------
0 | 50
5 | 50
(2 rows)
EXECUTE prepared_task_tracker_non_partition_column_select(60);
key | value
---------------------------------------------------------------------
0 | 60
6 | 60
(2 rows)
EXECUTE prepared_task_tracker_non_partition_column_select(67);
key | value
---------------------------------------------------------------------
(0 rows)
PREPARE prepared_task_tracker_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.key = $1 OR
prepare_table.value = 10
ORDER BY
key,
value;
EXECUTE prepared_task_tracker_partition_column_select(1);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
1 |
(3 rows)
EXECUTE prepared_task_tracker_partition_column_select(2);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
2 | 20
2 |
(4 rows)
EXECUTE prepared_task_tracker_partition_column_select(3);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
3 | 30
3 |
(4 rows)
EXECUTE prepared_task_tracker_partition_column_select(4);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
4 | 40
4 |
(4 rows)
EXECUTE prepared_task_tracker_partition_column_select(5);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
5 | 50
5 |
(4 rows)
EXECUTE prepared_task_tracker_partition_column_select(6);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
6 | 60
6 |
(4 rows)
EXECUTE prepared_task_tracker_partition_column_select(7);
key | value
---------------------------------------------------------------------
0 | 10
1 | 10
7 |
(3 rows)
-- check updates
PREPARE prepared_partition_parameter_update(int, int) AS
UPDATE prepare_table SET value = $2 WHERE key = $1;
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_partition_parameter_update(1, 11);
EXECUTE prepared_partition_parameter_update(2, 21);
EXECUTE prepared_partition_parameter_update(3, 31);
EXECUTE prepared_partition_parameter_update(4, 41);
EXECUTE prepared_partition_parameter_update(5, 51);
EXECUTE prepared_partition_parameter_update(6, 61);
EXECUTE prepared_partition_parameter_update(7, 71);
PREPARE prepared_non_partition_parameter_update(int, int) AS
UPDATE prepare_table SET value = $2 WHERE key = 0 AND value = $1;
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_non_partition_parameter_update(10, 12);
EXECUTE prepared_non_partition_parameter_update(20, 22);
EXECUTE prepared_non_partition_parameter_update(30, 32);
EXECUTE prepared_non_partition_parameter_update(40, 42);
EXECUTE prepared_non_partition_parameter_update(50, 52);
EXECUTE prepared_non_partition_parameter_update(60, 62);
EXECUTE prepared_non_partition_parameter_update(70, 72);
-- check after updates
SELECT * FROM prepare_table ORDER BY key, value;
key | value
---------------------------------------------------------------------
0 | 12
0 | 22
0 | 32
0 | 42
0 | 52
0 | 62
0 |
0 |
0 |
0 |
0 |
0 |
0 |
1 | 11
1 | 11
2 | 21
2 | 21
3 | 31
3 | 31
4 | 41
4 | 41
5 | 51
5 | 51
6 | 61
6 | 61
7 | 71
(26 rows)
-- check deletes
PREPARE prepared_partition_parameter_delete(int, int) AS
DELETE FROM prepare_table WHERE key = $1 AND value = $2;
EXECUTE prepared_partition_parameter_delete(1, 11);
EXECUTE prepared_partition_parameter_delete(2, 21);
EXECUTE prepared_partition_parameter_delete(3, 31);
EXECUTE prepared_partition_parameter_delete(4, 41);
EXECUTE prepared_partition_parameter_delete(5, 51);
EXECUTE prepared_partition_parameter_delete(6, 61);
EXECUTE prepared_partition_parameter_delete(7, 71);
PREPARE prepared_non_partition_parameter_delete(int) AS
DELETE FROM prepare_table WHERE key = 0 AND value = $1;
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_non_partition_parameter_delete(12);
EXECUTE prepared_non_partition_parameter_delete(22);
EXECUTE prepared_non_partition_parameter_delete(32);
EXECUTE prepared_non_partition_parameter_delete(42);
EXECUTE prepared_non_partition_parameter_delete(52);
EXECUTE prepared_non_partition_parameter_delete(62);
EXECUTE prepared_non_partition_parameter_delete(72);
-- check after deletes
SELECT * FROM prepare_table ORDER BY key, value;
key | value
---------------------------------------------------------------------
0 |
0 |
0 |
0 |
0 |
0 |
0 |
(7 rows)

View File

@ -0,0 +1,47 @@
SET search_path TO "prepared statements";
-- test function evaluation with parameters in an expression
PREPARE prepared_function_evaluation_insert(int) AS
INSERT INTO prepare_func_table (key, value1) VALUES ($1+1, 0*random());
-- execute 7 times to trigger prepared statement usage
EXECUTE prepared_function_evaluation_insert(1);
EXECUTE prepared_function_evaluation_insert(2);
EXECUTE prepared_function_evaluation_insert(3);
EXECUTE prepared_function_evaluation_insert(4);
EXECUTE prepared_function_evaluation_insert(5);
EXECUTE prepared_function_evaluation_insert(6);
EXECUTE prepared_function_evaluation_insert(7);
SELECT key, value1 FROM prepare_func_table ORDER BY key, value1;
key | value1
---------------------------------------------------------------------
2 | 0
3 | 0
4 | 0
5 | 0
6 | 0
7 | 0
8 | 0
(7 rows)
TRUNCATE prepare_func_table;
-- make it a bit harder: parameter wrapped in a function call
PREPARE wrapped_parameter_evaluation(text,text[]) AS
INSERT INTO prepare_func_table (key,value2) VALUES ($1,array_to_string($2,''));
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
SELECT key, value2 FROM prepare_func_table ORDER BY key, value2;
key | value2
---------------------------------------------------------------------
key | value
key | value
key | value
key | value
key | value
key | value
key | value
(7 rows)

View File

@ -0,0 +1,52 @@
SET search_path TO "prepared statements";
PREPARE prepared_relabel_insert(varchar) AS
INSERT INTO text_partition_column_table VALUES ($1, 1);
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
SELECT key, value FROM text_partition_column_table ORDER BY key;
key | value
---------------------------------------------------------------------
test | 1
test | 1
test | 1
test | 1
test | 1
test | 1
test | 1
(7 rows)
PREPARE prepared_coercion_to_domain_insert(text) AS
INSERT INTO domain_partition_column_table VALUES ($1, 1);
EXECUTE prepared_coercion_to_domain_insert('test-1');
EXECUTE prepared_coercion_to_domain_insert('test-2');
EXECUTE prepared_coercion_to_domain_insert('test-3');
EXECUTE prepared_coercion_to_domain_insert('test-4');
EXECUTE prepared_coercion_to_domain_insert('test-5');
EXECUTE prepared_coercion_to_domain_insert('test-6');
EXECUTE prepared_coercion_to_domain_insert('test-7');
PREPARE FOO AS INSERT INTO http_request (
site_id, ingest_time, url, request_country,
ip_address, status_code, response_time_msec
) VALUES (
1, clock_timestamp(), 'http://example.com/path', 'USA',
inet '88.250.10.123', 200, 10
);
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
SELECT count(distinct ingest_time) FROM http_request WHERE site_id = 1;
count
---------------------------------------------------------------------
8
(1 row)

View File

@ -0,0 +1,100 @@
CREATE SCHEMA "prepared statements";
SET search_path TO "prepared statements";
CREATE TABLE repartition_prepared_test (a int, b int);
SELECT create_distributed_table('repartition_prepared_test', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO repartition_prepared_test SELECT i%2, i%3 FROM generate_series(0,24)i;
-- create a custom type which also exists on worker nodes
CREATE TYPE test_composite_type AS (
i integer,
i2 integer
);
CREATE TABLE router_executor_table (
id bigint NOT NULL,
comment varchar(20),
stats test_composite_type
);
SELECT create_distributed_table('router_executor_table', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- test router executor with prepare statements
CREATE TABLE prepare_table (
key int,
value int
);
SELECT create_distributed_table('prepare_table','key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Testing parameters + function evaluation
CREATE TABLE prepare_func_table (
key text,
value1 int,
value2 text,
value3 timestamptz DEFAULT now()
);
SELECT create_distributed_table('prepare_func_table', 'key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- test function evaluation with parameters in an expression
PREPARE prepared_function_evaluation_insert(int) AS
INSERT INTO prepare_func_table (key, value1) VALUES ($1+1, 0*random());
-- Text columns can give issues when there is an implicit cast from varchar
CREATE TABLE text_partition_column_table (
key text NOT NULL,
value int
);
SELECT create_distributed_table('text_partition_column_table', 'key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Domain type columns can give issues
-- and we use offset to prevent output diverging
CREATE DOMAIN test_key AS text CHECK(VALUE ~ '^test-\d$');
SELECT run_command_on_workers($$
CREATE DOMAIN "prepared statements".test_key AS text CHECK(VALUE ~ '^test-\d$')
$$) OFFSET 10000;
run_command_on_workers
---------------------------------------------------------------------
(0 rows)
CREATE TABLE domain_partition_column_table (
key test_key NOT NULL,
value int
);
SELECT create_distributed_table('domain_partition_column_table', 'key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- verify we re-evaluate volatile functions every time
CREATE TABLE http_request (
site_id INT,
ingest_time TIMESTAMPTZ DEFAULT now(),
url TEXT,
request_country TEXT,
ip_address TEXT,
status_code INT,
response_time_msec INT
);
SELECT create_distributed_table('http_request', 'site_id');
create_distributed_table
---------------------------------------------------------------------
(1 row)

View File

@ -0,0 +1,65 @@
SET search_path to "te;'st", public;
INSERT INTO dist SELECT *,* FROM generate_series(1,100);
INSERT INTO dist2 SELECT *,* FROM generate_series(1,100);
INSERT INTO dist2 SELECT *,* FROM generate_series(1,100);
INSERT INTO ref SELECT *,* FROM generate_series(1,100);
SELECT COUNT(*) FROM dist join ref USING (a);
count
---------------------------------------------------------------------
100
(1 row)
SELECT COUNT(*) FROM dist join ref USING (a) WHERE dist.a =5;
count
---------------------------------------------------------------------
1
(1 row)
SELECT COUNT(*) FROM dist as d1 join dist as d2 USING (a);
count
---------------------------------------------------------------------
100
(1 row)
SELECT COUNT(*) FROM dist as d1 join dist as d2 USING (a) WHERE d1.a =5;
count
---------------------------------------------------------------------
1
(1 row)
SELECT COUNT(*) FROM dist as d1 join dist2 as d2 USING (a);
count
---------------------------------------------------------------------
200
(1 row)
SELECT COUNT(*) FROM dist as d1 join dist2 as d2 USING (a) WHERE d1.a =5;
count
---------------------------------------------------------------------
2
(1 row)
SELECT COUNT(*) FROM dist as d1 join dist as d2 on d1.a = d2.b;
count
---------------------------------------------------------------------
100
(1 row)
SELECT COUNT(*) FROM dist as d1 join dist as d2 on d1.a = d2.b WHERE d1.a =5;
count
---------------------------------------------------------------------
1
(1 row)
SELECT COUNT(*) FROM dist as d1 join dist2 as d2 on d1.a = d2.b;
count
---------------------------------------------------------------------
200
(1 row)
SELECT COUNT(*) FROM dist as d1 join dist2 as d2 on d1.a = d2.b WHERE d1.a =5;
count
---------------------------------------------------------------------
2
(1 row)

View File

@ -0,0 +1 @@
test: postgres

View File

@ -0,0 +1,186 @@
SET search_path to "ch benchmarks";
SET search_path to "ch benchmarks";
-- Query 1
SELECT
ol_number,
sum(ol_quantity) as sum_qty,
sum(ol_amount) as sum_amount,
avg(ol_quantity) as avg_qty,
avg(ol_amount) as avg_amount,
count(*) as count_order
FROM order_line
WHERE ol_delivery_d > '2007-01-02 00:00:00.000000'
GROUP BY ol_number
ORDER BY ol_number;
-- Query 2
SELECT
su_suppkey,
su_name,
n_name,
i_id,
i_name,
su_address,
su_phone,
su_comment
FROM
item,
supplier,
stock,
nation,
region,
(SELECT
s_i_id AS m_i_id,
min(s_quantity) as m_s_quantity
FROM
stock,
supplier,
nation,
region
WHERE mod((s_w_id*s_i_id),10000)=su_suppkey
AND su_nationkey=n_nationkey
AND n_regionkey=r_regionkey
AND r_name LIKE 'Europ%'
GROUP BY s_i_id) m
WHERE i_id = s_i_id
AND mod((s_w_id * s_i_id), 10000) = su_suppkey
AND su_nationkey = n_nationkey
AND n_regionkey = r_regionkey
AND i_data LIKE '%b'
AND r_name LIKE 'Europ%'
AND i_id = m_i_id
AND s_quantity = m_s_quantity
ORDER BY
n_name,
su_name,
i_id;
-- Query 3
SELECT
ol_o_id,
ol_w_id,
ol_d_id,
sum(ol_amount) AS revenue,
o_entry_d
FROM
customer,
new_order,
oorder,
order_line
WHERE c_state LIKE 'C%' -- used to ba A%, but C% works with our small data
AND c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND no_w_id = o_w_id
AND no_d_id = o_d_id
AND no_o_id = o_id
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND o_entry_d > '2007-01-02 00:00:00.000000'
GROUP BY
ol_o_id,
ol_w_id,
ol_d_id,
o_entry_d
ORDER BY
revenue DESC,
o_entry_d;
-- Query 4
SELECT
o_ol_cnt,
count(*) as order_count
FROM
oorder
WHERE o_entry_d >= '2007-01-02 00:00:00.000000'
AND o_entry_d < '2012-01-02 00:00:00.000000'
AND exists (SELECT *
FROM order_line
WHERE o_id = ol_o_id
AND o_w_id = ol_w_id
AND o_d_id = ol_d_id
AND ol_delivery_d >= o_entry_d)
GROUP BY o_ol_cnt
ORDER BY o_ol_cnt;
-- Query 5
SELECT
n_name,
sum(ol_amount) AS revenue
FROM
customer,
oorder,
order_line,
stock,
supplier,
nation,
region
WHERE c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND ol_o_id = o_id
AND ol_w_id = o_w_id
AND ol_d_id=o_d_id
AND ol_w_id = s_w_id
AND ol_i_id = s_i_id
AND mod((s_w_id * s_i_id),10000) = su_suppkey
-- our dataset does not have the supplier in the same nation as the customer causing this
-- join to filter out all the data. We verify later on that we can actually perform an
-- ascii(substr(c_state,1,1)) == reference table column join later on so it should not
-- matter we skip this filter here.
--AND ascii(substr(c_state,1,1)) = su_nationkey
AND su_nationkey = n_nationkey
AND n_regionkey = r_regionkey
AND r_name = 'Europe'
AND o_entry_d >= '2007-01-02 00:00:00.000000'
GROUP BY n_name
ORDER BY revenue DESC;
-- Query 6
SELECT
sum(ol_amount) AS revenue
FROM order_line
WHERE ol_delivery_d >= '1999-01-01 00:00:00.000000'
AND ol_delivery_d < '2020-01-01 00:00:00.000000'
AND ol_quantity BETWEEN 1 AND 100000;
-- Query 7
SELECT
su_nationkey as supp_nation,
substr(c_state,1,1) as cust_nation,
extract(year from o_entry_d) as l_year,
sum(ol_amount) as revenue
FROM
supplier,
stock,
order_line,
oorder,
customer,
nation n1,
nation n2
WHERE ol_supply_w_id = s_w_id
AND ol_i_id = s_i_id
AND mod((s_w_id * s_i_id), 10000) = su_suppkey
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND su_nationkey = n1.n_nationkey
AND ascii(substr(c_state,1,1)) = n2.n_nationkey
AND (
(n1.n_name = 'Germany' AND n2.n_name = 'Cambodia')
OR (n1.n_name = 'Cambodia' AND n2.n_name = 'Germany')
)
AND ol_delivery_d BETWEEN '2007-01-02 00:00:00.000000' AND '2012-01-02 00:00:00.000000'
GROUP BY
su_nationkey,
substr(c_state,1,1),
extract(year from o_entry_d)
ORDER BY
su_nationkey,
cust_nation,
l_year;

View File

@ -0,0 +1,187 @@
SET search_path to "ch benchmarks";
-- Query 8
SELECT
extract(year from o_entry_d) as l_year,
sum(case when n2.n_name = 'Germany' then ol_amount else 0 end) / sum(ol_amount) as mkt_share
FROM
item,
supplier,
stock,
order_line,
oorder,
customer,
nation n1,
nation n2,
region
WHERE i_id = s_i_id
AND ol_i_id = s_i_id
AND ol_supply_w_id = s_w_id
AND mod((s_w_id * s_i_id),10000) = su_suppkey
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND n1.n_nationkey = ascii(substr(c_state,1,1))
AND n1.n_regionkey = r_regionkey
AND ol_i_id < 1000
AND r_name = 'Europe'
AND su_nationkey = n2.n_nationkey
AND o_entry_d BETWEEN '2007-01-02 00:00:00.000000' AND '2012-01-02 00:00:00.000000'
AND i_data LIKE '%b'
AND i_id = ol_i_id
GROUP BY extract(YEAR FROM o_entry_d)
ORDER BY l_year;
-- Query 9
SELECT
n_name,
extract(year from o_entry_d) as l_year,
sum(ol_amount) as sum_profit
FROM
item,
stock,
supplier,
order_line,
oorder,
nation
WHERE ol_i_id = s_i_id
AND ol_supply_w_id = s_w_id
AND mod((s_w_id * s_i_id), 10000) = su_suppkey
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND ol_i_id = i_id
AND su_nationkey = n_nationkey
AND i_data LIKE '%b' -- this used to be %BB but that will not work with our small dataset
GROUP BY
n_name,
extract(YEAR FROM o_entry_d)
ORDER BY
n_name,
l_year DESC;
-- Query 10
SELECT
c_id,
c_last,
sum(ol_amount) AS revenue,
c_city,
c_phone,
n_name
FROM
customer,
oorder,
order_line,
nation
WHERE c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND o_entry_d >= '2007-01-02 00:00:00.000000'
AND o_entry_d <= ol_delivery_d
AND n_nationkey = ascii(substr(c_state,1,1))
GROUP BY
c_id,
c_last,
c_city,
c_phone,
n_name
ORDER BY revenue DESC;
-- Query 11
SELECT
s_i_id,
sum(s_order_cnt) AS ordercount
FROM
stock,
supplier,
nation
WHERE mod((s_w_id * s_i_id),10000) = su_suppkey
AND su_nationkey = n_nationkey
AND n_name = 'Germany'
GROUP BY s_i_id
HAVING sum(s_order_cnt) >
(SELECT sum(s_order_cnt) * .005
FROM
stock,
supplier,
nation
WHERE mod((s_w_id * s_i_id),10000) = su_suppkey
AND su_nationkey = n_nationkey
AND n_name = 'Germany')
ORDER BY ordercount DESC;
-- Query 12
SELECT
o_ol_cnt,
sum(case when o_carrier_id = 1 or o_carrier_id = 2 then 1 else 0 end) as high_line_count,
sum(case when o_carrier_id <> 1 and o_carrier_id <> 2 then 1 else 0 end) as low_line_count
FROM
oorder,
order_line
WHERE ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
AND o_entry_d <= ol_delivery_d
AND ol_delivery_d < '2020-01-01 00:00:00.000000'
GROUP BY o_ol_cnt
ORDER BY o_ol_cnt;
-- Query 13
SELECT
c_count,
count(*) AS custdist
FROM (SELECT
c_id,
count(o_id)
FROM customer
LEFT OUTER JOIN oorder ON (
c_w_id = o_w_id
AND c_d_id = o_d_id
AND c_id = o_c_id
AND o_carrier_id > 8)
GROUP BY c_id) AS c_orders (c_id, c_count)
GROUP BY c_count
ORDER BY
custdist DESC,
c_count DESC;
-- Query 14
SELECT
100.00 * sum(CASE WHEN i_data LIKE 'PR%' THEN ol_amount ELSE 0 END) / (1+sum(ol_amount)) AS promo_revenue
FROM
order_line,
item
WHERE ol_i_id = i_id
AND ol_delivery_d >= '2007-01-02 00:00:00.000000'
AND ol_delivery_d < '2020-01-02 00:00:00.000000';
-- Query 15
WITH revenue (supplier_no, total_revenue) AS (
SELECT
mod((s_w_id * s_i_id),10000) AS supplier_no,
sum(ol_amount) AS total_revenue
FROM
order_line,
stock
WHERE ol_i_id = s_i_id
AND ol_supply_w_id = s_w_id
AND ol_delivery_d >= '2007-01-02 00:00:00.000000'
GROUP BY mod((s_w_id * s_i_id),10000))
SELECT
su_suppkey,
su_name,
su_address,
su_phone,
total_revenue
FROM
supplier,
revenue
WHERE su_suppkey = supplier_no
AND total_revenue = (SELECT max(total_revenue) FROM revenue)
ORDER BY su_suppkey;

View File

@ -0,0 +1,27 @@
SET search_path to "ch benchmarks";
-- Subquery + repartion is supported when it is an IN query where the subquery
-- returns unique results (because it's converted to an INNER JOIN)
select s_i_id
from stock, order_line
where
s_i_id in (select i_id from item)
AND s_i_id = ol_i_id
order by s_i_id;
select su_name, su_address
from supplier, nation
where su_suppkey in
(select mod(s_i_id * s_w_id, 10000)
from stock, order_line
where s_i_id in
(select i_id
from item
where i_data like 'ab%')
and ol_i_id=s_i_id
and ol_delivery_d > '2010-05-23 12:00:00'
group by s_i_id, s_w_id, s_quantity
having 2*s_quantity > sum(ol_quantity))
and su_nationkey = n_nationkey
and n_name = 'Germany'
order by su_name;

View File

@ -0,0 +1,112 @@
SET search_path to "ch benchmarks";
--Q16
SELECT
i_name,
substr(i_data, 1, 3) AS brand,
i_price,
count(DISTINCT (mod((s_w_id * s_i_id),10000))) AS supplier_cnt
FROM
stock,
item
WHERE i_id = s_i_id
AND i_data NOT LIKE 'zz%'
AND (mod((s_w_id * s_i_id),10000) NOT IN
(SELECT su_suppkey
FROM supplier
WHERE su_comment LIKE '%bad%'))
GROUP BY
i_name,
substr(i_data, 1, 3),
i_price
ORDER BY supplier_cnt DESC;
--Q17
SELECT
sum(ol_amount) / 2.0 AS avg_yearly
FROM
order_line,
(SELECT
i_id,
avg(ol_quantity) AS a
FROM
item,
order_line
WHERE i_data LIKE '%b'
AND ol_i_id = i_id
GROUP BY i_id) t
WHERE ol_i_id = t.i_id;
-- this filter was at the end causing the dataset to be empty. it should not have any
-- influence on how the query gets planned so I removed the clause
--AND ol_quantity < t.a;
-- Query 18
SELECT
c_last,
c_id o_id,
o_entry_d,
o_ol_cnt,
sum(ol_amount)
FROM
customer,
oorder,
order_line
WHERE c_id = o_c_id
AND c_w_id = o_w_id
AND c_d_id = o_d_id
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_o_id = o_id
GROUP BY
o_id,
o_w_id,
o_d_id,
c_id,
c_last,
o_entry_d,
o_ol_cnt
HAVING sum(ol_amount) > 5 -- was 200, but thats too big for the dataset
ORDER BY
sum(ol_amount) DESC,
o_entry_d;
-- Query 19
SELECT
sum(ol_amount) AS revenue
FROM
order_line,
item
WHERE ( ol_i_id = i_id
AND i_data LIKE '%a'
AND ol_quantity >= 1
AND ol_quantity <= 10
AND i_price BETWEEN 1 AND 400000
AND ol_w_id IN (1,2,3))
OR ( ol_i_id = i_id
AND i_data LIKE '%b'
AND ol_quantity >= 1
AND ol_quantity <= 10
AND i_price BETWEEN 1 AND 400000
AND ol_w_id IN (1,2,4))
OR ( ol_i_id = i_id
AND i_data LIKE '%c'
AND ol_quantity >= 1
AND ol_quantity <= 10
AND i_price BETWEEN 1 AND 400000
AND ol_w_id IN (1,5,3));
select su_name, su_address
from supplier, nation
where su_suppkey in
(select mod(s_i_id * s_w_id, 10000)
from stock, order_line
where s_i_id in
(select i_id
from item)
and ol_i_id=s_i_id
and ol_delivery_d > '2010-05-23 12:00:00'
group by s_i_id, s_w_id, s_quantity
having 2*s_quantity > sum(ol_quantity))
and su_nationkey = n_nationkey
and n_name = 'GERMANY'
order by su_name;

View File

@ -0,0 +1,52 @@
SET search_path to "ch benchmarks";
-- Query 21
-- DATA SET DOES NOT COVER THIS QUERY
SELECT
su_name,
count(*) AS numwait
FROM
supplier,
order_line l1,
oorder,
stock,
nation
WHERE ol_o_id = o_id
AND ol_w_id = o_w_id
AND ol_d_id = o_d_id
AND ol_w_id = s_w_id
AND ol_i_id = s_i_id
AND mod((s_w_id * s_i_id),10000) = su_suppkey
AND l1.ol_delivery_d > o_entry_d
AND NOT exists (SELECT *
FROM order_line l2
WHERE l2.ol_o_id = l1.ol_o_id
AND l2.ol_w_id = l1.ol_w_id
AND l2.ol_d_id = l1.ol_d_id
AND l2.ol_delivery_d > l1.ol_delivery_d)
AND su_nationkey = n_nationkey
AND n_name = 'Germany'
GROUP BY su_name
ORDER BY
numwait desc,
su_name;
-- Query 22
-- DATA SET DOES NOT COVER THIS QUERY
SELECT
substr(c_state,1,1) AS country,
count(*) AS numcust,
sum(c_balance) AS totacctbal
FROM customer
WHERE substr(c_phone,1,1) in ('1','2','3','4','5','6','7')
AND c_balance > (SELECT avg(c_BALANCE)
FROM customer
WHERE c_balance > 0.00
AND substr(c_phone,1,1) in ('1','2','3','4','5','6','7'))
AND NOT exists (SELECT *
FROM oorder
WHERE o_c_id = c_id
AND o_w_id = c_w_id
AND o_d_id = c_d_id)
GROUP BY substr(c_state,1,1)
ORDER BY substr(c_state,1,1);

View File

@ -0,0 +1,122 @@
SET search_path to "ch benchmarks";
-- Query 20
SELECT
su_name,
su_address
FROM
supplier,
nation
WHERE su_suppkey in
(SELECT
mod(s_i_id * s_w_id, 10000)
FROM
stock,
order_line
WHERE s_i_id IN
(SELECT i_id
FROM item
WHERE i_data LIKE 'co%')
AND ol_i_id = s_i_id
AND ol_delivery_d > '2008-05-23 12:00:00' -- was 2010, but our order is in 2008
GROUP BY s_i_id, s_w_id, s_quantity
HAVING 2*s_quantity > sum(ol_quantity))
AND su_nationkey = n_nationkey
AND n_name = 'Germany'
ORDER BY su_name;
-- Multiple subqueries are supported IN and a NOT IN when no repartition join
-- is necessary and the IN subquery returns unique results
select s_i_id
from stock
where
s_i_id in (select i_id from item)
AND s_i_id not in (select i_im_id from item);
select s_i_id, sum(s_order_cnt) as ordercount
from stock
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
ORDER BY s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select true)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select true)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock
group by s_i_id
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having (select true)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
group by s_i_id
having (select false)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select true)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select false)
order by s_i_id;
select s_i_id, sum(s_order_cnt) as ordercount
from stock s
group by s_i_id
having (select true)
order by s_i_id;

View File

@ -0,0 +1,177 @@
CREATE SCHEMA "ch benchmarks";
SET search_path to "ch benchmarks";
CREATE TABLE order_line (
ol_w_id int NOT NULL,
ol_d_id int NOT NULL,
ol_o_id int NOT NULL,
ol_number int NOT NULL,
ol_i_id int NOT NULL,
ol_delivery_d timestamp NULL DEFAULT NULL,
ol_amount decimal(6,2) NOT NULL,
ol_supply_w_id int NOT NULL,
ol_quantity decimal(2,0) NOT NULL,
ol_dist_info char(24) NOT NULL,
PRIMARY KEY (ol_w_id,ol_d_id,ol_o_id,ol_number)
);
CREATE TABLE new_order (
no_w_id int NOT NULL,
no_d_id int NOT NULL,
no_o_id int NOT NULL,
PRIMARY KEY (no_w_id,no_d_id,no_o_id)
);
CREATE TABLE stock (
s_w_id int NOT NULL,
s_i_id int NOT NULL,
s_quantity decimal(4,0) NOT NULL,
s_ytd decimal(8,2) NOT NULL,
s_order_cnt int NOT NULL,
s_remote_cnt int NOT NULL,
s_data varchar(50) NOT NULL,
s_dist_01 char(24) NOT NULL,
s_dist_02 char(24) NOT NULL,
s_dist_03 char(24) NOT NULL,
s_dist_04 char(24) NOT NULL,
s_dist_05 char(24) NOT NULL,
s_dist_06 char(24) NOT NULL,
s_dist_07 char(24) NOT NULL,
s_dist_08 char(24) NOT NULL,
s_dist_09 char(24) NOT NULL,
s_dist_10 char(24) NOT NULL,
PRIMARY KEY (s_w_id,s_i_id)
);
CREATE TABLE oorder (
o_w_id int NOT NULL,
o_d_id int NOT NULL,
o_id int NOT NULL,
o_c_id int NOT NULL,
o_carrier_id int DEFAULT NULL,
o_ol_cnt decimal(2,0) NOT NULL,
o_all_local decimal(1,0) NOT NULL,
o_entry_d timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
PRIMARY KEY (o_w_id,o_d_id,o_id),
UNIQUE (o_w_id,o_d_id,o_c_id,o_id)
);
CREATE TABLE history (
h_c_id int NOT NULL,
h_c_d_id int NOT NULL,
h_c_w_id int NOT NULL,
h_d_id int NOT NULL,
h_w_id int NOT NULL,
h_date timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
h_amount decimal(6,2) NOT NULL,
h_data varchar(24) NOT NULL
);
CREATE TABLE customer (
c_w_id int NOT NULL,
c_d_id int NOT NULL,
c_id int NOT NULL,
c_discount decimal(4,4) NOT NULL,
c_credit char(2) NOT NULL,
c_last varchar(16) NOT NULL,
c_first varchar(16) NOT NULL,
c_credit_lim decimal(12,2) NOT NULL,
c_balance decimal(12,2) NOT NULL,
c_ytd_payment float NOT NULL,
c_payment_cnt int NOT NULL,
c_delivery_cnt int NOT NULL,
c_street_1 varchar(20) NOT NULL,
c_street_2 varchar(20) NOT NULL,
c_city varchar(20) NOT NULL,
c_state char(2) NOT NULL,
c_zip char(9) NOT NULL,
c_phone char(16) NOT NULL,
c_since timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
c_middle char(2) NOT NULL,
c_data varchar(500) NOT NULL,
PRIMARY KEY (c_w_id,c_d_id,c_id)
);
CREATE TABLE district (
d_w_id int NOT NULL,
d_id int NOT NULL,
d_ytd decimal(12,2) NOT NULL,
d_tax decimal(4,4) NOT NULL,
d_next_o_id int NOT NULL,
d_name varchar(10) NOT NULL,
d_street_1 varchar(20) NOT NULL,
d_street_2 varchar(20) NOT NULL,
d_city varchar(20) NOT NULL,
d_state char(2) NOT NULL,
d_zip char(9) NOT NULL,
PRIMARY KEY (d_w_id,d_id)
);
CREATE TABLE item (
i_id int NOT NULL,
i_name varchar(24) NOT NULL,
i_price decimal(5,2) NOT NULL,
i_data varchar(50) NOT NULL,
i_im_id int NOT NULL,
PRIMARY KEY (i_id)
);
CREATE TABLE warehouse (
w_id int NOT NULL,
w_ytd decimal(12,2) NOT NULL,
w_tax decimal(4,4) NOT NULL,
w_name varchar(10) NOT NULL,
w_street_1 varchar(20) NOT NULL,
w_street_2 varchar(20) NOT NULL,
w_city varchar(20) NOT NULL,
w_state char(2) NOT NULL,
w_zip char(9) NOT NULL,
PRIMARY KEY (w_id)
);
CREATE TABLE region (
r_regionkey int not null,
r_name char(55) not null,
r_comment char(152) not null,
PRIMARY KEY ( r_regionkey )
);
CREATE TABLE nation (
n_nationkey int not null,
n_name char(25) not null,
n_regionkey int not null,
n_comment char(152) not null,
PRIMARY KEY ( n_nationkey )
);
CREATE TABLE supplier (
su_suppkey int not null,
su_name char(25) not null,
su_address varchar(40) not null,
su_nationkey int not null,
su_phone char(15) not null,
su_acctbal numeric(12,2) not null,
su_comment char(101) not null,
PRIMARY KEY ( su_suppkey )
);
SELECT create_distributed_table('order_line','ol_w_id');
SELECT create_distributed_table('new_order','no_w_id');
SELECT create_distributed_table('stock','s_w_id');
SELECT create_distributed_table('oorder','o_w_id');
SELECT create_distributed_table('history','h_w_id');
SELECT create_distributed_table('customer','c_w_id');
SELECT create_distributed_table('district','d_w_id');
SELECT create_distributed_table('warehouse','w_id');
SELECT create_reference_table('item');
SELECT create_reference_table('region');
SELECT create_reference_table('nation');
SELECT create_reference_table('supplier');
TRUNCATE order_line, new_order, stock, oorder, history, customer, district, warehouse, item, region, nation, supplier; -- for easy copy in development
INSERT INTO supplier SELECT c, 'abc', 'def', c, 'ghi', c, 'jkl' FROM generate_series(0,10) AS c;
INSERT INTO new_order SELECT c, c, c FROM generate_series(0,10) AS c;
INSERT INTO stock SELECT c,c,c,c,c,c, 'abc','abc','abc','abc','abc','abc','abc','abc','abc','abc','abc' FROM generate_series(1,3) AS c;
INSERT INTO stock SELECT c, 5000,c,c,c,c, 'abc','abc','abc','abc','abc','abc','abc','abc','abc','abc','abc' FROM generate_series(1,3) AS c; -- mod(2*5000,10000) == 0
INSERT INTO order_line SELECT c, c, c, c, c, '2008-10-17 00:00:00.000000', c, c, c, 'abc' FROM generate_series(0,10) AS c;
INSERT INTO oorder SELECT c, c, c, c, c, 1, 1, '2008-10-17 00:00:00.000000' FROM generate_series(0,10) AS c;
INSERT INTO customer SELECT c, c, c, 0, 'XX', 'John', 'Doe', 1000, 0, 0, c, c, 'Name', 'Street', 'Some City', 'CA', '12345', '+1 000 0000000', '2007-01-02 00:00:00.000000', 'NA', 'nothing special' FROM generate_series(0,10) AS c;
INSERT INTO item SELECT c, 'Keyboard', 50, 'co b', c FROM generate_series(0,10) AS c; --co% and %b filters all around
INSERT INTO region VALUES
(1, 'Not Europe', 'Big'),
(2, 'Europe', 'Big');
INSERT INTO nation VALUES
(1, 'United States', 1, 'Also Big'),
(4, 'The Netherlands', 2, 'Flat'),
(9, 'Germany', 2, 'Germany must be in here for Q7'),
(67, 'Cambodia', 2, 'I don''t understand how we got from California to Cambodia but I will take it, it also is not in Europe, but we need it to be for Q8');

View File

@ -0,0 +1,11 @@
CREATE SCHEMA "te;'st";
SET search_path to "te;'st", public;
CREATE TABLE dist(a int, b int);
SELECT create_distributed_table('dist', 'a');
CREATE TABLE dist2(a int, b int);
SELECT create_distributed_table('dist2', 'a');
CREATE table ref(a int, b int);
SELECT create_reference_table('ref');

View File

@ -0,0 +1,252 @@
SET search_path TO "distributed planning";
-- Confirm the basics work
INSERT INTO test VALUES (1, 2), (3, 4), (5, 6), (2, 7), (4, 5);
INSERT INTO test VALUES (6, 7);
SELECT * FROM test WHERE x = 1 ORDER BY y, x;
SELECT t1.x, t2.y FROM test t1 JOIN test t2 USING(x) WHERE t1.x = 1 AND t2.x = 1 ORDER BY t2.y, t1.x;
SELECT * FROM test WHERE x = 1 OR x = 2 ORDER BY y, x;
SELECT count(*) FROM test;
SELECT * FROM test ORDER BY x;
WITH cte_1 AS (UPDATE test SET y = y - 1 RETURNING *) SELECT * FROM cte_1 ORDER BY 1,2;
-- observe that there is a conflict and the following query does nothing
INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT DO NOTHING RETURNING *;
-- same as the above with different syntax
INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT (part_key) DO NOTHING RETURNING *;
-- again the same query with another syntax
INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO NOTHING RETURNING *;
BEGIN;
-- force local execution if possible
SELECT count(*) FROM upsert_test WHERE part_key = 1;
-- multi-shard pushdown query that goes through local execution
INSERT INTO upsert_test (part_key, other_col) SELECT part_key, other_col FROM upsert_test ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO NOTHING RETURNING *;
INSERT INTO upsert_test (part_key, other_col) SELECT part_key, other_col FROM upsert_test ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO UPDATE SET other_col=EXCLUDED.other_col + 1 RETURNING *;
-- multi-shard pull-to-coordinator query that goes through local execution
INSERT INTO upsert_test (part_key, other_col) SELECT part_key, other_col FROM upsert_test LIMIT 100 ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO NOTHING RETURNING *;
INSERT INTO upsert_test (part_key, other_col) SELECT part_key, other_col FROM upsert_test LIMIT 100 ON CONFLICT ON CONSTRAINT upsert_test_part_key_key DO UPDATE SET other_col=EXCLUDED.other_col + 1 RETURNING *;
COMMIT;
BEGIN;
INSERT INTO test SELECT i,i from generate_series(0,1000)i;
-- only pulls 1 row, should not hit the limit
WITH cte_1 AS (SELECT * FROM test LIMIT 1) SELECT count(*) FROM cte_1;
-- cte_1 only pulls 1 row, but cte_2 all rows
WITH cte_1 AS (SELECT * FROM test LIMIT 1),
cte_2 AS (SELECT * FROM test OFFSET 0)
SELECT count(*) FROM cte_1, cte_2;
ROLLBACK;
-- single shard and multi-shard delete
-- inside a transaction block
BEGIN;
DELETE FROM test WHERE y = 5;
INSERT INTO test VALUES (4, 5);
DELETE FROM test WHERE x = 1;
INSERT INTO test VALUES (1, 2);
COMMIT;
-- basic view queries
CREATE VIEW simple_view AS
SELECT count(*) as cnt FROM test t1 JOIN test t2 USING (x);
SELECT * FROM simple_view;
SELECT * FROM simple_view, test WHERE test.x = simple_view.cnt;
BEGIN;
COPY test(x) FROM STDIN;
1
2
3
4
5
6
7
8
9
10
\.
SELECT count(*) FROM test;
COPY (SELECT count(DISTINCT x) FROM test) TO STDOUT;
INSERT INTO test SELECT i,i FROM generate_series(0,100)i;
ROLLBACK;
-- prepared statements with custom types
PREPARE single_node_prepare_p1(int, int, new_type) AS
INSERT INTO test_2 VALUES ($1, $2, $3);
EXECUTE single_node_prepare_p1(1, 1, (95, 'citus9.5')::new_type);
EXECUTE single_node_prepare_p1(2 ,2, (94, 'citus9.4')::new_type);
EXECUTE single_node_prepare_p1(3 ,2, (93, 'citus9.3')::new_type);
EXECUTE single_node_prepare_p1(4 ,2, (92, 'citus9.2')::new_type);
EXECUTE single_node_prepare_p1(5 ,2, (91, 'citus9.1')::new_type);
EXECUTE single_node_prepare_p1(6 ,2, (90, 'citus9.0')::new_type);
EXECUTE single_node_prepare_p1(6 ,2, (90, 'citus9.0')::new_type);
PREPARE use_local_query_cache(int) AS SELECT count(*) FROM test_2 WHERE x = $1;
EXECUTE use_local_query_cache(1);
EXECUTE use_local_query_cache(1);
EXECUTE use_local_query_cache(1);
EXECUTE use_local_query_cache(1);
EXECUTE use_local_query_cache(1);
EXECUTE use_local_query_cache(1);
EXECUTE use_local_query_cache(1);
BEGIN;
INSERT INTO test_2 VALUES (7 ,2, (83, 'citus8.3')::new_type);
SAVEPOINT s1;
INSERT INTO test_2 VALUES (9 ,1, (82, 'citus8.2')::new_type);
SAVEPOINT s2;
ROLLBACK TO SAVEPOINT s1;
SELECT * FROM test_2 WHERE z = (83, 'citus8.3')::new_type OR z = (82, 'citus8.2')::new_type;
RELEASE SAVEPOINT s1;
COMMIT;
SELECT * FROM test_2 WHERE z = (83, 'citus8.3')::new_type OR z = (82, 'citus8.2')::new_type;
WITH cte_1 AS (SELECT * FROM test_2) SELECT * FROM cte_1 ORDER BY 1,2;
-- final query is router query
WITH cte_1 AS (SELECT * FROM test_2) SELECT * FROM cte_1, test_2 WHERE test_2.x = cte_1.x AND test_2.x = 7 ORDER BY 1,2;
-- final query is a distributed query
WITH cte_1 AS (SELECT * FROM test_2) SELECT * FROM cte_1, test_2 WHERE test_2.x = cte_1.x AND test_2.y != 2 ORDER BY 1,2;
SELECT count(DISTINCT x) FROM test;
SELECT count(DISTINCT y) FROM test;
-- query pushdown should work
SELECT
*
FROM
(SELECT x, count(*) FROM test_2 GROUP BY x) as foo,
(SELECT x, count(*) FROM test_2 GROUP BY x) as bar
WHERE
foo.x = bar.x
ORDER BY 1 DESC, 2 DESC, 3 DESC, 4 DESC
LIMIT 1;
-- Check repartion joins are supported
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x, t2.x, t1.y, t2.y;
-- INSERT SELECT router
BEGIN;
INSERT INTO test(x, y) SELECT x, y FROM test WHERE x = 1;
SELECT count(*) from test;
ROLLBACK;
-- INSERT SELECT pushdown
BEGIN;
INSERT INTO test(x, y) SELECT x, y FROM test;
SELECT count(*) from test;
ROLLBACK;
-- INSERT SELECT analytical query
BEGIN;
INSERT INTO test(x, y) SELECT count(x), max(y) FROM test;
SELECT count(*) from test;
ROLLBACK;
-- INSERT SELECT repartition
BEGIN;
INSERT INTO test(x, y) SELECT y, x FROM test;
SELECT count(*) from test;
ROLLBACK;
-- INSERT SELECT from reference table into distributed
BEGIN;
INSERT INTO test(x, y) SELECT a, b FROM ref;
SELECT count(*) from test;
ROLLBACK;
-- INSERT SELECT from local table into distributed
BEGIN;
INSERT INTO test(x, y) SELECT c, d FROM local;
SELECT count(*) from test;
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO ref(a, b) SELECT x, y FROM test;
SELECT count(*) from ref;
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO ref(a, b) SELECT c, d FROM local;
SELECT count(*) from ref;
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO local(c, d) SELECT x, y FROM test;
SELECT count(*) from local;
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO local(c, d) SELECT a, b FROM ref;
SELECT count(*) from local;
ROLLBACK;
-- Confirm that dummy placements work
SELECT count(*) FROM test WHERE false;
SELECT count(*) FROM test WHERE false GROUP BY GROUPING SETS (x,y);
SELECT count(*) FROM test;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO ref(a, b) SELECT x, y FROM test;
SELECT count(*) from ref;
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO ref(a, b) SELECT c, d FROM local;
SELECT count(*) from ref;
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO local(c, d) SELECT x, y FROM test;
SELECT count(*) from local;
ROLLBACK;
-- INSERT SELECT from distributed table to local table
BEGIN;
INSERT INTO local(c, d) SELECT a, b FROM ref;
SELECT count(*) from local;
ROLLBACK;
-- query fails on the shards should be handled
-- nicely
\set VERBOSITY terse
SELECT x/0 FROM test;
SELECT count(DISTINCT row(key, value)) FROM non_binary_copy_test;
SELECT count(*), event FROM date_part_table GROUP BY event ORDER BY count(*) DESC, event DESC LIMIT 5;
SELECT count(*), event FROM date_part_table WHERE user_id = 1 GROUP BY event ORDER BY count(*) DESC, event DESC LIMIT 5;
SELECT count(*), t1.event FROM date_part_table t1 JOIN date_part_table USING (user_id) WHERE t1.user_id = 1 GROUP BY t1.event ORDER BY count(*) DESC, t1.event DESC LIMIT 5;
SELECT count(*), event FROM date_part_table WHERE event_time > '2020-01-05' GROUP BY event ORDER BY count(*) DESC, event DESC LIMIT 5;
SELECT count(*), event FROM date_part_table WHERE user_id = 12 AND event_time = '2020-01-12 12:00:00' GROUP BY event ORDER BY count(*) DESC, event DESC LIMIT 5;
SELECT count(*), t1.event FROM date_part_table t1 JOIN date_part_table t2 USING (user_id) WHERE t1.user_id = 1 AND t2.event_time > '2020-01-03' GROUP BY t1.event ORDER BY count(*) DESC, t1.event DESC LIMIT 5;

View File

@ -0,0 +1,52 @@
CREATE SCHEMA "distributed planning";
SET search_path TO "distributed planning";
CREATE TABLE
date_part_table (event_time timestamp, event int, user_id int)
partition by range (event_time);
SELECT create_distributed_table('date_part_table', 'user_id');
-- an unnamed index
CREATE INDEX ON date_part_table(user_id, event_time);
--create named and long index with parameters
CREATE INDEX event_idx ON date_part_table(event, event_time, user_id, user_id, event_time);
SELECT create_time_partitions(table_name:='date_part_table',
partition_interval:= '1 week',
end_at:= '2020-01-20',
start_from:='2020-01-01');
INSERT INTO date_part_table
SELECT '2020-01-01'::timestamp + '3 hours'::interval * i, i, i % 20 FROM generate_series(0,100)i;
CREATE TABLE test(x int, y int);
SELECT create_distributed_table('test','x');
CREATE TYPE new_type AS (n int, m text);
CREATE TABLE test_2(x int, y int, z new_type);
SELECT create_distributed_table('test_2','x');
CREATE TABLE ref(a int, b int);
SELECT create_reference_table('ref');
CREATE TABLE local(c int, d int);
select citus_add_local_table_to_metadata('local');
CREATE TABLE non_binary_copy_test (key int PRIMARY KEY, value new_type);
SELECT create_distributed_table('non_binary_copy_test', 'key');
INSERT INTO non_binary_copy_test SELECT i, (i, 'citus9.5')::new_type FROM generate_series(0,1000)i;
-- Test upsert with constraint
CREATE TABLE upsert_test
(
part_key int UNIQUE,
other_col int,
third_col int
);
-- distribute the table
SELECT create_distributed_table('upsert_test', 'part_key');
-- do a regular insert
INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1), (2, 2) RETURNING *;

View File

@ -0,0 +1,130 @@
SET search_path TO local_shard_execution_dropped_column;
prepare p1(int) as insert into t1(a,c) VALUES (5,$1) ON CONFLICT (c) DO NOTHING;
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
execute p1(8);
prepare p2(int) as SELECT count(*) FROM t1 WHERE c = $1 GROUP BY c;
execute p2(8);
execute p2(8);
execute p2(8);
execute p2(8);
execute p2(8);
execute p2(8);
execute p2(8);
execute p2(8);
execute p2(8);
execute p2(8);
prepare p3(int) as INSERT INTO t1(a,c) VALUES (5, $1), (6, $1), (7, $1),(5, $1), (6, $1), (7, $1) ON CONFLICT DO NOTHING;
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
execute p3(8);
prepare p4(int) as UPDATE t1 SET a = a + 1 WHERE c = $1;
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
execute p4(8);
-- https://github.com/citusdata/citus/issues/5355
-- prepare p5(int) as INSERT INTO t1(a,c) VALUES (15, $1) ON CONFLICT (c) DO UPDATE SET a=EXCLUDED.a + 10 RETURNING *;
-- execute p5(18);
-- execute p5(19);
-- execute p5(20);
-- execute p5(21);
-- execute p5(22);
-- execute p5(23);
-- execute p5(24);
-- execute p5(25);
-- execute p5(26);
-- execute p5(27);
-- execute p5(28);
-- execute p5(29);
-- show that all the tables prune to the same shard for the same distribution key
WITH
sensors_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors', 3)),
sensors_2000_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2000', 3)),
sensors_2001_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2001', 3)),
sensors_2002_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2002', 3)),
sensors_2003_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2003', 3)),
sensors_2004_shardid AS (SELECT * FROM get_shard_id_for_distribution_column('sensors_2004', 3)),
all_shardids AS (SELECT * FROM sensors_shardid UNION SELECT * FROM sensors_2000_shardid UNION
SELECT * FROM sensors_2001_shardid UNION SELECT * FROM sensors_2002_shardid
UNION SELECT * FROM sensors_2003_shardid UNION SELECT * FROM sensors_2004_shardid)
-- it is zero for PG only tests, and 1 for Citus
SELECT count(DISTINCT row(shardminvalue, shardmaxvalue)) <= 1 FROM pg_dist_shard WHERE shardid IN (SELECT * FROM all_shardids);
INSERT INTO sensors VALUES (3, '2000-02-02', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2000-01-01', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2001-01-01', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2002-01-01', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2003-01-01', row_to_json(row(1)));
INSERT INTO sensors VALUES (3, '2004-01-01', row_to_json(row(1)));
SELECT count(*) FROM sensors WHERE measureid = 3 AND eventdatetime = '2000-02-02';
SELECT count(*) FROM sensors_2000 WHERE measureid = 3;
SELECT count(*) FROM sensors_2001 WHERE measureid = 3;
SELECT count(*) FROM sensors_2002 WHERE measureid = 3;
SELECT count(*) FROM sensors_2003 WHERE measureid = 3;
-- multi-shard queries
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors;
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2000;
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2001;
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2002;
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2003;
SELECT count(DISTINCT row(measureid, eventdatetime, measure_data)) FROM sensors_2004;
-- execute 7 times to make sure it is re-cached
-- prepared statements should work fine even after columns are dropped
PREPARE drop_col_prepare_insert(int, date, jsonb) AS INSERT INTO sensors (measureid, eventdatetime, measure_data) VALUES ($1, $2, $3);
PREPARE drop_col_prepare_select(int, date) AS SELECT count(*) FROM sensors WHERE measureid = $1 AND eventdatetime = $2;
PREPARE drop_col_prepare_mshard_select(date) AS SELECT count(*) FROM sensors WHERE eventdatetime = $1;
EXECUTE drop_col_prepare_insert(3, '2000-10-01', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(3, '2001-10-01', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(3, '2002-10-01', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(3, '2003-10-01', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(3, '2003-10-02', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(4, '2003-10-03', row_to_json(row(1)));
EXECUTE drop_col_prepare_insert(5, '2003-10-04', row_to_json(row(1)));
EXECUTE drop_col_prepare_select(3, '2000-10-01');
EXECUTE drop_col_prepare_select(3, '2001-10-01');
EXECUTE drop_col_prepare_select(3, '2002-10-01');
EXECUTE drop_col_prepare_select(3, '2003-10-01');
EXECUTE drop_col_prepare_select(3, '2003-10-02');
EXECUTE drop_col_prepare_select(4, '2003-10-03');
EXECUTE drop_col_prepare_select(5, '2003-10-04');
EXECUTE drop_col_prepare_mshard_select('2000-10-01');
EXECUTE drop_col_prepare_mshard_select('2000-10-01');
EXECUTE drop_col_prepare_mshard_select('2001-10-01');
EXECUTE drop_col_prepare_mshard_select('2002-10-01');
EXECUTE drop_col_prepare_mshard_select('2002-10-01');
EXECUTE drop_col_prepare_mshard_select('2003-10-01');
EXECUTE drop_col_prepare_mshard_select('2003-10-01');
EXECUTE drop_col_prepare_mshard_select('2004-01-01');

View File

@ -0,0 +1,62 @@
CREATE SCHEMA local_shard_execution_dropped_column;
SET search_path TO local_shard_execution_dropped_column;
CREATE TABLE t1 (a int, b int, c int UNIQUE, d int, e int);
ALTER TABLE t1 DROP COLUMN e;
SELECT create_distributed_table('t1', 'c');
ALTER TABLE t1 DROP COLUMN b;
ALTER TABLE t1 DROP COLUMN d;
-- create a partitioned table with some columns that
-- are going to be dropped within the tests
CREATE TABLE sensors(
col_to_drop_0 text,
col_to_drop_1 text,
col_to_drop_2 date,
col_to_drop_3 inet,
col_to_drop_4 date,
measureid integer,
eventdatetime date,
measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data))
PARTITION BY RANGE(eventdatetime);
-- drop column even before attaching any partitions
ALTER TABLE sensors DROP COLUMN col_to_drop_1;
-- now attach the first partition and create the distributed table
CREATE TABLE sensors_2000 PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2001-01-01');
SELECT create_distributed_table('sensors', 'measureid');
-- drop another column before attaching another partition
-- with .. PARTITION OF .. syntax
ALTER TABLE sensors DROP COLUMN col_to_drop_0;
CREATE TABLE sensors_2001 PARTITION OF sensors FOR VALUES FROM ('2001-01-01') TO ('2002-01-01');
-- drop another column before attaching another partition
-- with ALTER TABLE .. ATTACH PARTITION
ALTER TABLE sensors DROP COLUMN col_to_drop_2;
CREATE TABLE sensors_2002(
col_to_drop_4 date, col_to_drop_3 inet, measureid integer, eventdatetime date, measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data));
ALTER TABLE sensors ATTACH PARTITION sensors_2002 FOR VALUES FROM ('2002-01-01') TO ('2003-01-01');
-- drop another column before attaching another partition
-- that is already distributed
ALTER TABLE sensors DROP COLUMN col_to_drop_3;
CREATE TABLE sensors_2003(
col_to_drop_4 date, measureid integer, eventdatetime date, measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data));
SELECT create_distributed_table('sensors_2003', 'measureid');
ALTER TABLE sensors ATTACH PARTITION sensors_2003 FOR VALUES FROM ('2003-01-01') TO ('2004-01-01');
CREATE TABLE sensors_2004(
col_to_drop_4 date, measureid integer NOT NULL, eventdatetime date NOT NULL, measure_data jsonb NOT NULL);
ALTER TABLE sensors ATTACH PARTITION sensors_2004 FOR VALUES FROM ('2004-01-01') TO ('2005-01-01');
ALTER TABLE sensors DROP COLUMN col_to_drop_4;
SELECT alter_table_set_access_method('sensors_2004', 'columnar');

View File

@ -0,0 +1,32 @@
CREATE SCHEMA "intermediate result pruning";
SET search_path TO "intermediate result pruning";
CREATE TABLE table_1 (key int, value text);
SELECT create_distributed_table('table_1', 'key');
CREATE TABLE table_2 (key int, value text);
SELECT create_distributed_table('table_2', 'key');
CREATE TABLE table_3 (key int, value text);
SELECT create_distributed_table('table_3', 'key');
CREATE TABLE ref_table (key int, value text);
SELECT create_reference_table('ref_table');
-- load some data
INSERT INTO table_1 VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4');
INSERT INTO table_2 VALUES (3, '3'), (4, '4'), (5, '5'), (6, '6');
INSERT INTO table_3 VALUES (3, '3'), (4, '4'), (5, '5'), (6, '6');
INSERT INTO ref_table VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6');
CREATE TABLE accounts (id text PRIMARY KEY);
CREATE TABLE stats (account_id text PRIMARY KEY, spent int);
SELECT create_distributed_table('accounts', 'id', colocate_with => 'none');
SELECT create_distributed_table('stats', 'account_id', colocate_with => 'accounts');
INSERT INTO accounts (id) VALUES ('foo');
INSERT INTO stats (account_id, spent) VALUES ('foo', 100);

View File

@ -0,0 +1,409 @@
SET search_path TO "intermediate result pruning";
-- a very basic case, where the intermediate result
-- should go to both workers
WITH some_values_1 AS MATERIALIZED
(SELECT key FROM table_1 WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
some_values_1 JOIN table_2 USING (key);
-- a very basic case, where the intermediate result
-- should only go to one worker because the final query is a router
-- we use random() to prevent postgres inline the CTE(s)
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
some_values_1 JOIN table_2 USING (key) WHERE table_2.key = 3;
-- a similar query, but with a reference table now
-- given that reference tables are replicated to all nodes
-- we have to broadcast to all nodes
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
some_values_1 JOIN ref_table USING (key);
-- a similar query as above, but this time use the CTE inside
-- another CTE
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1)
SELECT
count(*)
FROM
some_values_2 JOIN table_2 USING (key) WHERE table_2.key = 3;
-- the second CTE does a join with a distributed table
-- and the final query is a router query
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key))
SELECT
count(*)
FROM
some_values_2 JOIN table_2 USING (key) WHERE table_2.key = 3;
-- the first CTE is used both within second CTE and the final query
-- the second CTE does a join with a distributed table
-- and the final query is a router query
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key))
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key = 3;
-- the first CTE is used both within second CTE and the final query
-- the second CTE does a join with a distributed table but a router query on a worker
-- and the final query is another router query on another worker
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE table_2.key = 4)
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key = 4;
-- the first CTE is used both within second CTE and the final query
-- the second CTE does a join with a distributed table but a router query on a worker
-- and the final query is a router query on the same worker, so the first result is only
-- broadcasted to a single node
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE table_2.key = 3)
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key = 3;
-- the same query with the above, but the final query is hitting all shards
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key))
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key != 3;
-- even if we add a filter on the first query and make it a router query,
-- the first intermediate result still hits all workers because of the final
-- join is hitting all workers
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE table_2.key = 3)
SELECT
count(*)
FROM
(some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key) WHERE table_2.key != 4;
-- the reference table is joined with a distributed table and an intermediate
-- result, but the distributed table hits all shards, so the intermediate
-- result is sent to all nodes
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM ref_table WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
(some_values_1 JOIN ref_table USING (key)) JOIN table_2 USING (key);
-- similar query as above, but this time the whole query is a router
-- query, so no intermediate results
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM ref_table WHERE value IN ('3', '4'))
SELECT
count(*)
FROM
(some_values_1 JOIN ref_table USING (key)) JOIN table_2 USING (key) WHERE table_2.key = 4;
-- now, the second CTE has a single shard join with a distributed table
-- so the first CTE should only be broadcasted to that node
-- since the final query doesn't have a join, it should simply be broadcasted
-- to one node
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE key = 1)
SELECT
count(*)
FROM
some_values_2;
-- the same query inlined inside a CTE, and the final query has a
-- join with a distributed table
WITH top_cte as MATERIALIZED (
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE key = 4)
SELECT
DISTINCT key
FROM
some_values_2
)
SELECT
count(*)
FROM
top_cte JOIN table_2 USING (key);
-- very much the same query, but this time the top query is also a router query
-- on a single worker, so all intermediate results only hit a single node
WITH top_cte as MATERIALIZED (
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE key = 1)
SELECT
DISTINCT key
FROM
some_values_2
)
SELECT
count(*)
FROM
top_cte JOIN table_2 USING (key) WHERE table_2.key = 2;
-- some_values_1 is first used by a single shard-query, and than with a multi-shard
-- CTE, finally a cartesian product join
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('6', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1 JOIN table_2 USING (key) WHERE key = 4),
some_values_3 AS MATERIALIZED
(SELECT key FROM (some_values_2 JOIN table_2 USING (key)) JOIN some_values_1 USING (key))
SELECT * FROM some_values_3 JOIN ref_table ON (true) ORDER BY 1,2,3;
-- join on intermediate results, so should only
-- go to a single node
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM table_2 WHERE value IN ('3', '4'))
SELECT count(*) FROM some_values_2 JOIN some_values_1 USING (key);
-- same query with WHERE false make sure that we're not broken
-- for such edge cases
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM table_2 WHERE value IN ('3', '4'))
SELECT count(*) FROM some_values_2 JOIN some_values_1 USING (key) WHERE false;
-- do not use some_values_2 at all, so only 2 intermediate results are
-- broadcasted
WITH some_values_1 AS MATERIALIZED
(SELECT key, random() FROM table_1 WHERE value IN ('3', '4')),
some_values_2 AS MATERIALIZED
(SELECT key, random() FROM some_values_1),
some_values_3 AS MATERIALIZED
(SELECT key, random() FROM some_values_1)
SELECT
count(*)
FROM
some_values_3;
-- lets have some deeper intermediate results
-- the inner most two results and the final query (which contains only intermediate results)
-- hitting single worker, others hitting all workers
-- (see below query where all intermediate results hit a single node)
SELECT count(*) FROM
(
SELECT avg(min::int) FROM
(
SELECT min(table_1.value) FROM
(
SELECT avg(value::int) as avg_ev_type FROM
(
SELECT max(value) as mx_val_1 FROM
(
SELECT avg(value::int) as avg FROM
(
SELECT cnt FROM
(
SELECT count(*) as cnt, value
FROM table_1
WHERE key = 1
GROUP BY value
) as level_1, table_1
WHERE table_1.key = level_1.cnt AND key = 3
) as level_2, table_2
WHERE table_2.key = level_2.cnt AND key = 5
GROUP BY level_2.cnt
) as level_3, table_1
WHERE value::numeric = level_3.avg AND key = 6
GROUP BY level_3.avg
) as level_4, table_2
WHERE level_4.mx_val_1::int = table_2.key
GROUP BY level_4.mx_val_1
) as level_5, table_1
WHERE level_5.avg_ev_type = table_1.key AND key > 111
GROUP BY level_5.avg_ev_type
) as level_6, table_1 WHERE table_1.key::int = level_6.min::int
GROUP BY table_1.value
) as bar;
-- the same query where all intermediate results hits one
-- worker because each and every query is a router query -- but on different nodes
SELECT count(*) FROM
(
SELECT avg(min::int) FROM
(
SELECT min(table_1.value) FROM
(
SELECT avg(value::int) as avg_ev_type FROM
(
SELECT max(value) as mx_val_1 FROM
(
SELECT avg(value::int) as avg FROM
(
SELECT cnt FROM
(
SELECT count(*) as cnt, value
FROM table_1
WHERE key = 1
GROUP BY value
) as level_1, table_1
WHERE table_1.key = level_1.cnt AND key = 3
) as level_2, table_2
WHERE table_2.key = level_2.cnt AND key = 5
GROUP BY level_2.cnt
) as level_3, table_1
WHERE value::numeric = level_3.avg AND key = 6
GROUP BY level_3.avg
) as level_4, table_2
WHERE level_4.mx_val_1::int = table_2.key AND table_2.key = 1
GROUP BY level_4.mx_val_1
) as level_5, table_1
WHERE level_5.avg_ev_type = table_1.key AND key = 111
GROUP BY level_5.avg_ev_type
) as level_6, table_1
WHERE table_1.key::int = level_6.min::int AND table_1.key = 4
GROUP BY table_1.value
) as bar;
-- sanity checks for set operations
-- the intermediate results should just hit a single worker
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2);
-- the intermediate results should just hit a single worker
WITH cte_1 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2)
),
cte_2 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 3)
INTERSECT
(SELECT key FROM table_1 WHERE key = 4)
)
SELECT * FROM cte_1
UNION
SELECT * FROM cte_2;
-- one final test with SET operations, where
-- we join the results with distributed tables
-- so cte_1 should hit all workers, but still the
-- others should hit single worker each
WITH cte_1 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2)
),
cte_2 AS MATERIALIZED
(
SELECT count(*) FROM table_1 JOIN cte_1 USING (key)
)
SELECT * FROM cte_2;
-- sanity checks for non-colocated subquery joins
-- the recursively planned subquery (bar) should hit all
-- nodes
SELECT
count(*)
FROM
(SELECT key, random() FROM table_1) as foo,
(SELECT key, random() FROM table_2) as bar
WHERE
foo.key != bar.key;
-- the recursively planned subquery (bar) should hit one
-- node because foo goes to a single node
SELECT
count(*)
FROM
(SELECT key, random() FROM table_1 WHERE key = 1) as foo,
(SELECT key, random() FROM table_2) as bar
WHERE
foo.key != bar.key;
SELECT *
FROM
(
WITH accounts_cte AS MATERIALIZED (
SELECT id AS account_id
FROM accounts
),
joined_stats_cte_1 AS MATERIALIZED (
SELECT spent, account_id
FROM stats
INNER JOIN accounts_cte USING (account_id)
),
joined_stats_cte_2 AS MATERIALIZED (
SELECT spent, account_id
FROM joined_stats_cte_1
INNER JOIN accounts_cte USING (account_id)
)
SELECT SUM(spent) OVER (PARTITION BY coalesce(account_id, NULL))
FROM accounts_cte
INNER JOIN joined_stats_cte_2 USING (account_id)
) inner_query;
-- Testing a having clause that could have been a where clause between a distributed table
-- and a reference table. This query was the cause for intermediate results not being
-- available during the replace of the planner for the master query with the standard
-- planner.
-- Since the having clause could have been a where clause the having clause on the grouping
-- on the coordinator is replaced with a Result node containing a One-time filter if the
-- having qual (one-time filter works because the query doesn't change with the tuples
-- returned from below).
SELECT count(*),
spent
FROM stats
GROUP BY 2
HAVING (
SELECT count(*)
FROM accounts
) > 0;

View File

@ -0,0 +1,128 @@
SET search_path TO "intermediate result pruning";
-- sanity checks for modification queries
-- select_data goes to a single node, because it is used in another subquery
-- raw_data is also the final router query, so hits a single shard
-- however, the subquery in WHERE clause of the DELETE query is broadcasted to all
-- nodes
BEGIN;
WITH select_data AS MATERIALIZED (
SELECT * FROM table_1
),
raw_data AS MATERIALIZED (
DELETE FROM table_2 WHERE key >= (SELECT min(key) FROM select_data WHERE key > 1) RETURNING *
)
SELECT * FROM raw_data ORDER BY 1,2;
ROLLBACK;
-- select_data goes to a single node, because it is used in another subquery
-- raw_data is also the final router query, so hits a single shard
-- however, the subquery in WHERE clause of the DELETE query is broadcasted to all
-- nodes
BEGIN;
WITH select_data AS MATERIALIZED (
SELECT * FROM table_1
),
raw_data AS MATERIALIZED (
DELETE FROM table_2 WHERE value::int >= (SELECT min(key) FROM select_data WHERE key > 1 + random()) RETURNING *
)
SELECT * FROM raw_data ORDER BY 1,2;
ROLLBACK;
-- now, we need only two intermediate results as the subquery in WHERE clause is
-- router plannable
BEGIN;
WITH select_data AS MATERIALIZED (
SELECT * FROM table_1
),
raw_data AS MATERIALIZED (
DELETE FROM table_2 WHERE value::int >= (SELECT min(key) FROM table_1 WHERE key > random()) AND key = 6 RETURNING *
)
SELECT * FROM raw_data ORDER BY 1,2;
ROLLBACK;
-- test with INSERT SELECT via coordinator
-- INSERT .. SELECT via coordinator that doesn't have any intermediate results
-- We use offset 1 to make sure the result needs to be pulled to the coordinator, offset 0 would be optimized away
BEGIN;
INSERT INTO table_1
SELECT * FROM table_2 OFFSET 1;
ROLLBACK;
-- INSERT .. SELECT via coordinator which has intermediate result,
-- and can be pruned to a single worker because the final query is on
-- single shard via filter in key
BEGIN;
INSERT INTO table_1
SELECT * FROM table_2 where value IN (SELECT value FROM table_1 WHERE random() > 1) AND key = 1;
ROLLBACK;
-- a similar query, with more complex subquery
BEGIN;
INSERT INTO table_1
SELECT * FROM table_2 where key = 1 AND
value::int IN
(WITH cte_1 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2)
),
cte_2 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 3)
INTERSECT
(SELECT key FROM table_1 WHERE key = 4)
)
SELECT * FROM cte_1
UNION
SELECT * FROM cte_2);
ROLLBACK;
-- same query, cte is on the FROM clause
-- and this time the final query (and top-level intermediate result)
-- hits all the shards because table_2.key != 1
BEGIN;
INSERT INTO table_1
SELECT table_2.* FROM table_2,
(WITH cte_1 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 1)
INTERSECT
(SELECT key FROM table_1 WHERE key = 2)
),
cte_2 AS MATERIALIZED
(
(SELECT key FROM table_1 WHERE key = 3)
INTERSECT
(SELECT key FROM table_1 WHERE key = 4)
)
SELECT * FROM cte_1
UNION
SELECT * FROM cte_2
) foo
where table_2.key != 1 AND
foo.key = table_2.value::int;
ROLLBACK;
BEGIN;
-- Insert..select is planned differently, make sure we have results everywhere.
-- We put the insert..select in a CTE here to prevent the CTE from being moved
-- into the select, which would follow the regular code path for select.
WITH stats AS MATERIALIZED (
SELECT count(key) m FROM table_3
),
inserts AS MATERIALIZED (
INSERT INTO table_2
SELECT key, count(*)
FROM table_1
WHERE key >= (SELECT m FROM stats)
GROUP BY key
HAVING count(*) < (SELECT m FROM stats)
LIMIT 1
RETURNING *
) SELECT count(*) FROM inserts;
ROLLBACK;

View File

@ -0,0 +1,29 @@
-- We override the distribute functions so that we can test postgres configs easily.
CREATE OR REPLACE FUNCTION pg_catalog.create_distributed_table(table_name regclass,
distribution_column text,
distribution_type citus.distribution_type DEFAULT 'hash',
colocate_with text DEFAULT 'default',
shard_count int DEFAULT NULL)
RETURNS void
LANGUAGE plpgsql
AS $function$
BEGIN
END;
$function$;
CREATE OR REPLACE FUNCTION pg_catalog.create_reference_table(table_name regclass)
RETURNS void
LANGUAGE plpgsql
AS $function$
BEGIN
END;
$function$;
CREATE OR REPLACE FUNCTION pg_catalog.citus_add_local_table_to_metadata(table_name regclass, cascade_via_foreign_keys boolean default false)
RETURNS void
LANGUAGE plpgsql
AS $function$
BEGIN
END;
$function$;

View File

@ -0,0 +1,19 @@
SET search_path TO "prepared statements";
PREPARE repartition_prepared(int) AS
SELECT
count(*)
FROM
repartition_prepared_test t1
JOIN
repartition_prepared_test t2
USING (b)
WHERE t1.a = $1;
EXECUTE repartition_prepared (1);
BEGIN;
-- CREATE TABLE ... AS EXECUTE prepared_statement tests
CREATE TEMP TABLE repartition_prepared_tmp AS EXECUTE repartition_prepared(1);
SELECT count(*) from repartition_prepared_tmp;
ROLLBACK;

View File

@ -0,0 +1,306 @@
SET search_path TO "prepared statements";
-- test parameterized inserts
PREPARE prepared_insert(varchar(20)) AS
INSERT INTO router_executor_table VALUES (1, $1, $2);
EXECUTE prepared_insert('comment-1', '(1, 10)');
EXECUTE prepared_insert('comment-2', '(2, 20)');
EXECUTE prepared_insert('comment-3', '(3, 30)');
EXECUTE prepared_insert('comment-4', '(4, 40)');
EXECUTE prepared_insert('comment-5', '(5, 50)');
EXECUTE prepared_insert('comment-6', '(6, 60)');
EXECUTE prepared_insert('comment-7', '(7, 67)');
-- to make this work, Citus adds the type casting for composite keys
-- during the deparsing
PREPARE prepared_custom_type_select(test_composite_type) AS
SELECT count(*) FROM router_executor_table WHERE id = 1 AND stats = $1;
EXECUTE prepared_custom_type_select('(1,10)');
EXECUTE prepared_custom_type_select('(2,20)');
EXECUTE prepared_custom_type_select('(3,30)');
EXECUTE prepared_custom_type_select('(4,40)');
EXECUTE prepared_custom_type_select('(5,50)');
EXECUTE prepared_custom_type_select('(6,60)');
EXECUTE prepared_custom_type_select('(7,67)');
EXECUTE prepared_custom_type_select('(7,67)');
-- test parameterized selects
PREPARE prepared_select(integer, integer) AS
SELECT count(*) FROM router_executor_table
WHERE id = 1 AND stats = ROW($1, $2)::test_composite_type;
EXECUTE prepared_select(1, 10);
EXECUTE prepared_select(2, 20);
EXECUTE prepared_select(3, 30);
EXECUTE prepared_select(4, 40);
EXECUTE prepared_select(5, 50);
EXECUTE prepared_select(6, 60);
EXECUTE prepared_select(7, 67);
EXECUTE prepared_select(7, 67);
-- Test that parameterized partition column for an insert is supported
PREPARE prepared_partition_column_insert(bigint) AS
INSERT INTO router_executor_table VALUES ($1, 'arsenous', '(1,10)');
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_partition_column_insert(1);
EXECUTE prepared_partition_column_insert(2);
EXECUTE prepared_partition_column_insert(3);
EXECUTE prepared_partition_column_insert(4);
EXECUTE prepared_partition_column_insert(5);
EXECUTE prepared_partition_column_insert(6);
EXECUTE prepared_partition_column_insert(7);
PREPARE prepared_no_parameter_insert AS
INSERT INTO prepare_table (key) VALUES (0);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
EXECUTE prepared_no_parameter_insert;
PREPARE prepared_single_parameter_insert(int) AS
INSERT INTO prepare_table (key) VALUES ($1);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_single_parameter_insert(1);
EXECUTE prepared_single_parameter_insert(2);
EXECUTE prepared_single_parameter_insert(3);
EXECUTE prepared_single_parameter_insert(4);
EXECUTE prepared_single_parameter_insert(5);
EXECUTE prepared_single_parameter_insert(6);
EXECUTE prepared_single_parameter_insert(7);
PREPARE prepared_double_parameter_insert(int, int) AS
INSERT INTO prepare_table (key, value) VALUES ($1, $2);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_double_parameter_insert(1, 10);
EXECUTE prepared_double_parameter_insert(2, 20);
EXECUTE prepared_double_parameter_insert(3, 30);
EXECUTE prepared_double_parameter_insert(4, 40);
EXECUTE prepared_double_parameter_insert(5, 50);
EXECUTE prepared_double_parameter_insert(6, 60);
EXECUTE prepared_double_parameter_insert(7, 70);
PREPARE prepared_multi_insert(int, int) AS
INSERT INTO prepare_table (key, value) VALUES ($1, $2), ($1 + 1, $2 + 10);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_multi_insert( 7, 70);
EXECUTE prepared_multi_insert( 9, 90);
EXECUTE prepared_multi_insert(11, 110);
EXECUTE prepared_multi_insert(13, 130);
EXECUTE prepared_multi_insert(15, 150);
EXECUTE prepared_multi_insert(17, 170);
EXECUTE prepared_multi_insert(19, 190);
PREPARE prepared_non_partition_parameter_insert(int) AS
INSERT INTO prepare_table (key, value) VALUES (0, $1);
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_non_partition_parameter_insert(10);
EXECUTE prepared_non_partition_parameter_insert(20);
EXECUTE prepared_non_partition_parameter_insert(30);
EXECUTE prepared_non_partition_parameter_insert(40);
EXECUTE prepared_non_partition_parameter_insert(50);
EXECUTE prepared_non_partition_parameter_insert(60);
EXECUTE prepared_non_partition_parameter_insert(70);
-- check inserted values
SELECT count(*) FROM prepare_table;
DELETE FROM prepare_table WHERE value >= 70;
-- check router executor select
PREPARE prepared_router_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.key = $1
ORDER BY
key,
value;
EXECUTE prepared_router_partition_column_select(1);
EXECUTE prepared_router_partition_column_select(2);
EXECUTE prepared_router_partition_column_select(3);
EXECUTE prepared_router_partition_column_select(4);
EXECUTE prepared_router_partition_column_select(5);
EXECUTE prepared_router_partition_column_select(6);
EXECUTE prepared_router_partition_column_select(7);
PREPARE prepared_router_non_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.key = 0 AND
prepare_table.value = $1
ORDER BY
key,
value;
EXECUTE prepared_router_non_partition_column_select(10);
EXECUTE prepared_router_non_partition_column_select(20);
EXECUTE prepared_router_non_partition_column_select(30);
EXECUTE prepared_router_non_partition_column_select(40);
EXECUTE prepared_router_non_partition_column_select(50);
EXECUTE prepared_router_non_partition_column_select(60);
EXECUTE prepared_router_non_partition_column_select(67);
-- check real-time executor
PREPARE prepared_real_time_non_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.value = $1
ORDER BY
key,
value;
EXECUTE prepared_real_time_non_partition_column_select(10);
EXECUTE prepared_real_time_non_partition_column_select(20);
EXECUTE prepared_real_time_non_partition_column_select(30);
EXECUTE prepared_real_time_non_partition_column_select(40);
EXECUTE prepared_real_time_non_partition_column_select(50);
EXECUTE prepared_real_time_non_partition_column_select(60);
EXECUTE prepared_real_time_non_partition_column_select(70);
PREPARE prepared_real_time_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.key = $1 OR
prepare_table.value = 10
ORDER BY
key,
value;
EXECUTE prepared_real_time_partition_column_select(1);
EXECUTE prepared_real_time_partition_column_select(2);
EXECUTE prepared_real_time_partition_column_select(3);
EXECUTE prepared_real_time_partition_column_select(4);
EXECUTE prepared_real_time_partition_column_select(5);
EXECUTE prepared_real_time_partition_column_select(6);
EXECUTE prepared_real_time_partition_column_select(7);
PREPARE prepared_task_tracker_non_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.value = $1
ORDER BY
key,
value;
EXECUTE prepared_task_tracker_non_partition_column_select(10);
EXECUTE prepared_task_tracker_non_partition_column_select(20);
EXECUTE prepared_task_tracker_non_partition_column_select(30);
EXECUTE prepared_task_tracker_non_partition_column_select(40);
EXECUTE prepared_task_tracker_non_partition_column_select(50);
EXECUTE prepared_task_tracker_non_partition_column_select(60);
EXECUTE prepared_task_tracker_non_partition_column_select(67);
PREPARE prepared_task_tracker_partition_column_select(int) AS
SELECT
prepare_table.key,
prepare_table.value
FROM
prepare_table
WHERE
prepare_table.key = $1 OR
prepare_table.value = 10
ORDER BY
key,
value;
EXECUTE prepared_task_tracker_partition_column_select(1);
EXECUTE prepared_task_tracker_partition_column_select(2);
EXECUTE prepared_task_tracker_partition_column_select(3);
EXECUTE prepared_task_tracker_partition_column_select(4);
EXECUTE prepared_task_tracker_partition_column_select(5);
EXECUTE prepared_task_tracker_partition_column_select(6);
EXECUTE prepared_task_tracker_partition_column_select(7);
-- check updates
PREPARE prepared_partition_parameter_update(int, int) AS
UPDATE prepare_table SET value = $2 WHERE key = $1;
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_partition_parameter_update(1, 11);
EXECUTE prepared_partition_parameter_update(2, 21);
EXECUTE prepared_partition_parameter_update(3, 31);
EXECUTE prepared_partition_parameter_update(4, 41);
EXECUTE prepared_partition_parameter_update(5, 51);
EXECUTE prepared_partition_parameter_update(6, 61);
EXECUTE prepared_partition_parameter_update(7, 71);
PREPARE prepared_non_partition_parameter_update(int, int) AS
UPDATE prepare_table SET value = $2 WHERE key = 0 AND value = $1;
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_non_partition_parameter_update(10, 12);
EXECUTE prepared_non_partition_parameter_update(20, 22);
EXECUTE prepared_non_partition_parameter_update(30, 32);
EXECUTE prepared_non_partition_parameter_update(40, 42);
EXECUTE prepared_non_partition_parameter_update(50, 52);
EXECUTE prepared_non_partition_parameter_update(60, 62);
EXECUTE prepared_non_partition_parameter_update(70, 72);
-- check after updates
SELECT * FROM prepare_table ORDER BY key, value;
-- check deletes
PREPARE prepared_partition_parameter_delete(int, int) AS
DELETE FROM prepare_table WHERE key = $1 AND value = $2;
EXECUTE prepared_partition_parameter_delete(1, 11);
EXECUTE prepared_partition_parameter_delete(2, 21);
EXECUTE prepared_partition_parameter_delete(3, 31);
EXECUTE prepared_partition_parameter_delete(4, 41);
EXECUTE prepared_partition_parameter_delete(5, 51);
EXECUTE prepared_partition_parameter_delete(6, 61);
EXECUTE prepared_partition_parameter_delete(7, 71);
PREPARE prepared_non_partition_parameter_delete(int) AS
DELETE FROM prepare_table WHERE key = 0 AND value = $1;
-- execute 6 times to trigger prepared statement usage
EXECUTE prepared_non_partition_parameter_delete(12);
EXECUTE prepared_non_partition_parameter_delete(22);
EXECUTE prepared_non_partition_parameter_delete(32);
EXECUTE prepared_non_partition_parameter_delete(42);
EXECUTE prepared_non_partition_parameter_delete(52);
EXECUTE prepared_non_partition_parameter_delete(62);
EXECUTE prepared_non_partition_parameter_delete(72);
-- check after deletes
SELECT * FROM prepare_table ORDER BY key, value;

View File

@ -0,0 +1,32 @@
SET search_path TO "prepared statements";
-- test function evaluation with parameters in an expression
PREPARE prepared_function_evaluation_insert(int) AS
INSERT INTO prepare_func_table (key, value1) VALUES ($1+1, 0*random());
-- execute 7 times to trigger prepared statement usage
EXECUTE prepared_function_evaluation_insert(1);
EXECUTE prepared_function_evaluation_insert(2);
EXECUTE prepared_function_evaluation_insert(3);
EXECUTE prepared_function_evaluation_insert(4);
EXECUTE prepared_function_evaluation_insert(5);
EXECUTE prepared_function_evaluation_insert(6);
EXECUTE prepared_function_evaluation_insert(7);
SELECT key, value1 FROM prepare_func_table ORDER BY key, value1;
TRUNCATE prepare_func_table;
-- make it a bit harder: parameter wrapped in a function call
PREPARE wrapped_parameter_evaluation(text,text[]) AS
INSERT INTO prepare_func_table (key,value2) VALUES ($1,array_to_string($2,''));
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
EXECUTE wrapped_parameter_evaluation('key', ARRAY['value']);
SELECT key, value2 FROM prepare_func_table ORDER BY key, value2;

View File

@ -0,0 +1,50 @@
SET search_path TO "prepared statements";
PREPARE prepared_relabel_insert(varchar) AS
INSERT INTO text_partition_column_table VALUES ($1, 1);
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
EXECUTE prepared_relabel_insert('test');
SELECT key, value FROM text_partition_column_table ORDER BY key;
PREPARE prepared_coercion_to_domain_insert(text) AS
INSERT INTO domain_partition_column_table VALUES ($1, 1);
EXECUTE prepared_coercion_to_domain_insert('test-1');
EXECUTE prepared_coercion_to_domain_insert('test-2');
EXECUTE prepared_coercion_to_domain_insert('test-3');
EXECUTE prepared_coercion_to_domain_insert('test-4');
EXECUTE prepared_coercion_to_domain_insert('test-5');
EXECUTE prepared_coercion_to_domain_insert('test-6');
EXECUTE prepared_coercion_to_domain_insert('test-7');
PREPARE FOO AS INSERT INTO http_request (
site_id, ingest_time, url, request_country,
ip_address, status_code, response_time_msec
) VALUES (
1, clock_timestamp(), 'http://example.com/path', 'USA',
inet '88.250.10.123', 200, 10
);
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
EXECUTE foo;
SELECT count(distinct ingest_time) FROM http_request WHERE site_id = 1;

View File

@ -0,0 +1,79 @@
CREATE SCHEMA "prepared statements";
SET search_path TO "prepared statements";
CREATE TABLE repartition_prepared_test (a int, b int);
SELECT create_distributed_table('repartition_prepared_test', 'a');
INSERT INTO repartition_prepared_test SELECT i%2, i%3 FROM generate_series(0,24)i;
-- create a custom type which also exists on worker nodes
CREATE TYPE test_composite_type AS (
i integer,
i2 integer
);
CREATE TABLE router_executor_table (
id bigint NOT NULL,
comment varchar(20),
stats test_composite_type
);
SELECT create_distributed_table('router_executor_table', 'id');
-- test router executor with prepare statements
CREATE TABLE prepare_table (
key int,
value int
);
SELECT create_distributed_table('prepare_table','key');
-- Testing parameters + function evaluation
CREATE TABLE prepare_func_table (
key text,
value1 int,
value2 text,
value3 timestamptz DEFAULT now()
);
SELECT create_distributed_table('prepare_func_table', 'key');
-- test function evaluation with parameters in an expression
PREPARE prepared_function_evaluation_insert(int) AS
INSERT INTO prepare_func_table (key, value1) VALUES ($1+1, 0*random());
-- Text columns can give issues when there is an implicit cast from varchar
CREATE TABLE text_partition_column_table (
key text NOT NULL,
value int
);
SELECT create_distributed_table('text_partition_column_table', 'key');
-- Domain type columns can give issues
-- and we use offset to prevent output diverging
CREATE DOMAIN test_key AS text CHECK(VALUE ~ '^test-\d$');
SELECT run_command_on_workers($$
CREATE DOMAIN "prepared statements".test_key AS text CHECK(VALUE ~ '^test-\d$')
$$) OFFSET 10000;
CREATE TABLE domain_partition_column_table (
key test_key NOT NULL,
value int
);
SELECT create_distributed_table('domain_partition_column_table', 'key');
-- verify we re-evaluate volatile functions every time
CREATE TABLE http_request (
site_id INT,
ingest_time TIMESTAMPTZ DEFAULT now(),
url TEXT,
request_country TEXT,
ip_address TEXT,
status_code INT,
response_time_msec INT
);
SELECT create_distributed_table('http_request', 'site_id');

View File

@ -0,0 +1,19 @@
SET search_path to "te;'st", public;
INSERT INTO dist SELECT *,* FROM generate_series(1,100);
INSERT INTO dist2 SELECT *,* FROM generate_series(1,100);
INSERT INTO dist2 SELECT *,* FROM generate_series(1,100);
INSERT INTO ref SELECT *,* FROM generate_series(1,100);
SELECT COUNT(*) FROM dist join ref USING (a);
SELECT COUNT(*) FROM dist join ref USING (a) WHERE dist.a =5;
SELECT COUNT(*) FROM dist as d1 join dist as d2 USING (a);
SELECT COUNT(*) FROM dist as d1 join dist as d2 USING (a) WHERE d1.a =5;
SELECT COUNT(*) FROM dist as d1 join dist2 as d2 USING (a);
SELECT COUNT(*) FROM dist as d1 join dist2 as d2 USING (a) WHERE d1.a =5;
SELECT COUNT(*) FROM dist as d1 join dist as d2 on d1.a = d2.b;
SELECT COUNT(*) FROM dist as d1 join dist as d2 on d1.a = d2.b WHERE d1.a =5;
SELECT COUNT(*) FROM dist as d1 join dist2 as d2 on d1.a = d2.b;
SELECT COUNT(*) FROM dist as d1 join dist2 as d2 on d1.a = d2.b WHERE d1.a =5;

View File

@ -0,0 +1 @@
test: sql_test

View File

@ -0,0 +1,7 @@
test: sql_test
test: prepared_statements_1 prepared_statements_2 prepared_statements_3
test: prepared_statements_4
test: ch_benchmarks_1 ch_benchmarks_2 ch_benchmarks_3
test: ch_benchmarks_4 ch_benchmarks_5 ch_benchmarks_6
test: intermediate_result_pruning_queries_1 intermediate_result_pruning_queries_2
test: dropped_columns_1 distributed_planning

View File

@ -1,110 +0,0 @@
# Upgrade Tests
## Postgres Upgrade Test
Postgres upgrade test is used for testing postgres version upgrade with citus installed.
Before running the script, make sure that:
- You have downloaded citus.
- You have two different postgres versions.
- Citus is installed to both of the postgres versions. For each postgres version:
- In citus source directory run:
```bash
make clean
./configure PG_CONFIG=<your path to postgres pg config>
PG_CONFIG=<your path to postgres pg config> make
sudo PG_CONFIG=<your path to postgres pg config> make install
```
Make sure you do this for both postgres versions, pg_config should be different for each postgres version.
- Install `pipenv` and run in `citus/src/test/regress`:
```bash
pipenv install
pipenv shell
```
- Finally run upgrade test in `citus/src/test/regress`:
```bash
pipenv run make check-pg-upgrade old-bindir=<old-bindir> new-bindir=<new-bindir>
```
To see full command list:
```bash
pipenv run upgrade/pg_upgrade_test.py -help
```
How the postgres upgrade test works:
- Temporary folder `tmp_upgrade` is created in `src/test/regress/`, if one exists it is removed first.
- Database is initialized and citus cluster is created(1 coordinator + 2 workers) with old postgres.
- `before_pg_upgrade_schedule` is run with `pg_regress`. This schedule sets up any
objects and data that will be tested for preservation after the upgrade. It
- `after_pg_upgrade_schedule` is run with `pg_regress` to verify that the output
of those tests is the same before the upgrade as after.
- `citus_prepare_pg_upgrade` is run in coordinators and workers.
- Old database is stopped.
- A new database is initialized with new postgres under `tmp_upgrade`.
- Postgres upgrade is performed.
- New database is started in both coordinators and workers.
- `citus_finish_pg_upgrade` is run in coordinators and workers to finalize the upgrade step.
- `after_pg_upgrade_schedule` is run with `pg_regress` to verify that the previously created tables, and data still exist. Router and realtime queries are used to verify this.
### Writing new PG upgrade tests
The main important thing is that we have `upgrade_{name}_before` and
`upgrade_{name}_after` tests. The `before` files are used to setup any objects
and data before the upgrade. The `after` tests shouldn't have any side effects
since they are run twice (once before and once after the upgrade).
Furthermore, anything that is basic Citus functionality should go in the
`upgrade_basic_before`/`upgrade_basic_after` tests. This test file is used
during PG upgrades and Citus upgrades. Any features that don't work in old Citus
versions should thus be added to their own file, because that file will then
only be run during PG versions.
## Citus Upgrade Test
Citus upgrade test is used for testing citus version upgrades from specific version to master. The purpose of this test is to ensure that a newly made change does not result in unexpected upgrade errors.
Currently the citus upgrade test assumes that:
- You have citus artifact tarballs, both for old version and master.
How the citus upgrade test work:
- The script takes `citus-pre-tar` and `citus-post-tar` which should contain citus artifacts.
- It installs the given citus version from `citus-pre-tar`.
- It creates a citus cluster(1 coordinator 2 workers).
- It reports the initial versions.
- It installs the checked out citus version from `citus-post-tar`.
- It restarts the database and runs `ALTER EXTENSION citus UPGRADE`.
- It runs `after_citus_upgrade` schedule to verify that the upgrade is successful.
- It stops the cluster.
Note that when the version of citus changes, we should update `MASTER_VERSION` with the new version of citus otherwise that will be outdated and it will fail.
There is a target for citus upgrade test. We run citus upgrade tests both in normal mode and in mixed mode. In mixed mode, we don't upgrade one of the workers. `'citus.enable_version_checks' : 'false'` is used to prevent citus from giving an error for mixed mode.
To see full command list:
```bash
pipenv run upgrade/citus_upgrade_test.py -help
```
In order to run citus upgrade tests locally you can use:
```bash
pipenv run make check-citus-upgrade-local citus-old-version=v8.0.0
```
For mixed mode:
```bash
pipenv run make check-citus-upgrade-mixed-local citus-old-version=v8.0.0
```

View File

@ -1,68 +0,0 @@
from os.path import expanduser
BEFORE_PG_UPGRADE_SCHEDULE = './before_pg_upgrade_schedule'
AFTER_PG_UPGRADE_SCHEDULE = './after_pg_upgrade_schedule'
AFTER_CITUS_UPGRADE_COORD_SCHEDULE = './after_citus_upgrade_coord_schedule'
BEFORE_CITUS_UPGRADE_COORD_SCHEDULE = './before_citus_upgrade_coord_schedule'
MIXED_BEFORE_CITUS_UPGRADE_SCHEDULE = './mixed_before_citus_upgrade_schedule'
MIXED_AFTER_CITUS_UPGRADE_SCHEDULE = './mixed_after_citus_upgrade_schedule'
MASTER = 'master'
# This should be updated when citus version changes
MASTER_VERSION = '11.0'
HOME = expanduser("~")
CITUS_VERSION_SQL = "SELECT extversion FROM pg_extension WHERE extname = 'citus';"
class CitusUpgradeConfig():
def __init__(self, arguments):
self.bindir = arguments['--bindir']
self.pre_tar_path = arguments['--citus-pre-tar']
self.post_tar_path = arguments['--citus-post-tar']
self.pg_srcdir = arguments['--pgxsdir']
self.temp_dir = './tmp_citus_upgrade'
self.datadir = self.temp_dir + '/data'
self.settings = {
'shared_preload_libraries': 'citus',
'citus.node_conninfo': 'sslmode=prefer',
'citus.enable_version_checks' : 'false'
}
self.mixed_mode = arguments['--mixed']
class PGUpgradeConfig():
def __init__(self, arguments):
self.old_bindir = arguments['--old-bindir']
self.new_bindir = arguments['--new-bindir']
self.pg_srcdir = arguments['--pgxsdir']
self.temp_dir = './tmp_upgrade'
self.old_datadir = self.temp_dir + '/oldData'
self.new_datadir = self.temp_dir + '/newData'
self.settings = {
'shared_preload_libraries': 'citus',
'citus.node_conninfo': 'sslmode=prefer'
}
USER = 'postgres'
DBNAME = 'postgres'
COORDINATOR_NAME = 'coordinator'
WORKER1 = 'worker1'
WORKER2 = 'worker2'
NODE_NAMES = [COORDINATOR_NAME, WORKER1, WORKER2]
COORDINATOR_PORT = 57635
WORKER1PORT = 57636
WORKER2PORT = 57637
WORKER_PORTS = [WORKER1PORT, WORKER2PORT]
NODE_PORTS = {
COORDINATOR_NAME: COORDINATOR_PORT,
WORKER1: WORKER1PORT,
WORKER2: WORKER2PORT,
}

View File

@ -1,89 +0,0 @@
#!/usr/bin/env python3
"""upgrade_test
Usage:
upgrade_test --old-bindir=<old-bindir> --new-bindir=<new-bindir> --pgxsdir=<pgxsdir>
Options:
--old-bindir=<old-bindir> The old PostgreSQL executable directory(ex: '~/.pgenv/pgsql-10.4/bin')
--new-bindir=<new-bindir> The new PostgreSQL executable directory(ex: '~/.pgenv/pgsql-11.3/bin')
--pgxsdir=<pgxsdir> Path to the PGXS directory(ex: ~/.pgenv/src/postgresql-11.3)
"""
from config import (
PGUpgradeConfig, USER, NODE_PORTS,
NODE_NAMES, DBNAME, COORDINATOR_NAME,
WORKER_PORTS, AFTER_PG_UPGRADE_SCHEDULE, BEFORE_PG_UPGRADE_SCHEDULE
)
from docopt import docopt
import utils
import atexit
import subprocess
import sys
import shutil
import os
import upgrade_common as common
def citus_prepare_pg_upgrade(pg_path):
for port in NODE_PORTS.values():
utils.psql(pg_path, port, "SELECT citus_prepare_pg_upgrade();")
def perform_postgres_upgrade(old_bindir, new_bindir, old_datadir, new_datadir):
for node_name in NODE_NAMES:
base_new_data_path = os.path.abspath(new_datadir)
base_old_data_path = os.path.abspath(old_datadir)
with utils.cd(base_new_data_path):
abs_new_data_path = os.path.join(base_new_data_path, node_name)
abs_old_data_path = os.path.join(base_old_data_path, node_name)
command = [
os.path.join(new_bindir, 'pg_upgrade'),
'--username', USER,
'--old-bindir', old_bindir,
'--new-bindir', new_bindir,
'--old-datadir', abs_old_data_path,
'--new-datadir', abs_new_data_path
]
subprocess.run(command, check=True)
def citus_finish_pg_upgrade(pg_path):
for port in NODE_PORTS.values():
utils.psql(pg_path, port, "SELECT citus_finish_pg_upgrade();")
def stop_all_databases(old_bindir, new_bindir, old_datadir, new_datadir):
common.stop_databases(old_bindir, old_datadir)
common.stop_databases(new_bindir, new_datadir)
def main(config):
common.initialize_temp_dir(config.temp_dir)
common.initialize_citus_cluster(config.old_bindir, config.old_datadir, config.settings)
common.run_pg_regress(config.old_bindir, config.pg_srcdir,
NODE_PORTS[COORDINATOR_NAME], BEFORE_PG_UPGRADE_SCHEDULE)
common.run_pg_regress(config.old_bindir, config.pg_srcdir,
NODE_PORTS[COORDINATOR_NAME], AFTER_PG_UPGRADE_SCHEDULE)
citus_prepare_pg_upgrade(config.old_bindir)
# prepare should be idempotent, calling it a second time should never fail.
citus_prepare_pg_upgrade(config.old_bindir)
common.stop_databases(config.old_bindir, config.old_datadir)
common.initialize_db_for_cluster(
config.new_bindir, config.new_datadir, config.settings)
perform_postgres_upgrade(
config.old_bindir, config.new_bindir, config.old_datadir, config.new_datadir)
common.start_databases(config.new_bindir, config.new_datadir)
citus_finish_pg_upgrade(config.new_bindir)
common.run_pg_regress(config.new_bindir, config.pg_srcdir,
NODE_PORTS[COORDINATOR_NAME], AFTER_PG_UPGRADE_SCHEDULE)
if __name__ == '__main__':
config = PGUpgradeConfig(docopt(__doc__, version='upgrade_test'))
atexit.register(stop_all_databases, config.old_bindir,
config.new_bindir, config.old_datadir, config.new_datadir)
main(config)

View File

@ -1,104 +0,0 @@
import os
import shutil
import sys
import subprocess
import utils
from config import NODE_NAMES, NODE_PORTS, COORDINATOR_NAME, USER, WORKER_PORTS, DBNAME
def initialize_temp_dir(temp_dir):
if os.path.exists(temp_dir):
shutil.rmtree(temp_dir)
os.mkdir(temp_dir)
# Give full access to TEMP_DIR so that postgres user can use it.
os.chmod(temp_dir, 0o777)
def initialize_temp_dir_if_not_exists(temp_dir):
if os.path.exists(temp_dir):
return
os.mkdir(temp_dir)
# Give full access to TEMP_DIR so that postgres user can use it.
os.chmod(temp_dir, 0o777)
def initialize_db_for_cluster(pg_path, rel_data_path, settings):
subprocess.run(['mkdir', rel_data_path], check=True)
for node_name in NODE_NAMES:
abs_data_path = os.path.abspath(os.path.join(rel_data_path, node_name))
command = [
os.path.join(pg_path, 'initdb'),
'--pgdata', abs_data_path,
'--username', USER
]
subprocess.run(command, check=True)
add_settings(abs_data_path, settings)
def add_settings(abs_data_path, settings):
conf_path = os.path.join(abs_data_path, 'postgresql.conf')
with open(conf_path, 'a') as conf_file:
for setting_key, setting_val in settings.items():
setting = "{setting_key} = \'{setting_val}\'\n".format(
setting_key=setting_key,
setting_val=setting_val)
conf_file.write(setting)
def start_databases(pg_path, rel_data_path):
for node_name in NODE_NAMES:
abs_data_path = os.path.abspath(os.path.join(rel_data_path, node_name))
command = [
os.path.join(pg_path, 'pg_ctl'), 'start',
'--pgdata', abs_data_path,
'-U', USER,
'-o', '-p {}'.format(NODE_PORTS[node_name]),
'--log', os.path.join(abs_data_path, 'logfile_' + node_name)
]
subprocess.run(command, check=True)
def create_citus_extension(pg_path):
for port in NODE_PORTS.values():
utils.psql(pg_path, port, "CREATE EXTENSION citus;")
def run_pg_regress(pg_path, pg_srcdir, port, schedule):
command = [
os.path.join(pg_srcdir, 'src/test/regress/pg_regress'),
'--port', str(port),
'--schedule', schedule,
'--bindir', pg_path,
'--user', USER,
'--dbname', DBNAME,
'--use-existing'
]
exit_code = subprocess.call(command)
subprocess.run('bin/copy_modified', check=True)
if exit_code != 0:
sys.exit(exit_code)
def add_workers(pg_path):
for port in WORKER_PORTS:
command = "SELECT * from master_add_node('localhost', {port});".format(
port=port)
utils.psql(pg_path, NODE_PORTS[COORDINATOR_NAME], command)
def stop_databases(pg_path, rel_data_path):
for node_name in NODE_NAMES:
abs_data_path = os.path.abspath(os.path.join(rel_data_path, node_name))
command = [
os.path.join(pg_path, 'pg_ctl'), 'stop',
'--pgdata', abs_data_path,
'-U', USER,
'-o', '-p {}'.format(NODE_PORTS[node_name]),
'--log', os.path.join(abs_data_path, 'logfile_' + node_name)
]
subprocess.call(command)
def initialize_citus_cluster(old_bindir, old_datadir, settings):
initialize_db_for_cluster(old_bindir, old_datadir, settings)
start_databases(old_bindir, old_datadir)
create_citus_extension(old_bindir)
add_workers(old_bindir)