diff --git a/.circleci/config.yml b/.circleci/config.yml index afc416040..95fb34696 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -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 diff --git a/src/test/regress/.gitignore b/src/test/regress/.gitignore index 7eba53c6f..0458dc3ad 100644 --- a/src/test/regress/.gitignore +++ b/src/test/regress/.gitignore @@ -6,6 +6,7 @@ /tmp_upgrade/ /tmp_citus_upgrade/ /tmp_citus_tarballs/ +/tmp_citus_test/ /build/ /results/ /log/ diff --git a/src/test/regress/Makefile b/src/test/regress/Makefile index 685e84d2b..68adf6b28 100644 --- a/src/test/regress/Makefile +++ b/src/test/regress/Makefile @@ -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 diff --git a/src/test/regress/bin/copy_modified_wrapper b/src/test/regress/bin/copy_modified_wrapper new file mode 100755 index 000000000..b1c528e47 --- /dev/null +++ b/src/test/regress/bin/copy_modified_wrapper @@ -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 diff --git a/src/test/regress/citus_tests/__init__.py b/src/test/regress/citus_tests/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/src/test/regress/citus_tests/arbitrary_configs/README.md b/src/test/regress/citus_tests/arbitrary_configs/README.md new file mode 100644 index 000000000..d0b3c3e10 --- /dev/null +++ b/src/test/regress/citus_tests/arbitrary_configs/README.md @@ -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`. diff --git a/src/test/regress/citus_tests/arbitrary_configs/__init__.py b/src/test/regress/citus_tests/arbitrary_configs/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py b/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py new file mode 100755 index 000000000..f1992dc1f --- /dev/null +++ b/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py @@ -0,0 +1,239 @@ +#!/usr/bin/env python3 + +"""citus_arbitrary_configs +Usage: + citus_arbitrary_configs --bindir= --pgxsdir= --parallel= --configs= --seed= [--base] + +Options: + --bindir= The PostgreSQL executable directory(ex: '~/.pgenv/pgsql-11.3/bin') + --pgxsdir= Path to the PGXS directory(ex: ~/.pgenv/src/postgresql-11.3) + --parallel= how many configs to run in parallel + --configs= the config names to run + --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) diff --git a/src/test/regress/citus_tests/common.py b/src/test/regress/citus_tests/common.py new file mode 100644 index 000000000..39d721c14 --- /dev/null +++ b/src/test/regress/citus_tests/common.py @@ -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() diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py new file mode 100644 index 000000000..7d30cffdf --- /dev/null +++ b/src/test/regress/citus_tests/config.py @@ -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 diff --git a/src/test/regress/citus_tests/upgrade/README.md b/src/test/regress/citus_tests/upgrade/README.md new file mode 100644 index 000000000..1efd4e91d --- /dev/null +++ b/src/test/regress/citus_tests/upgrade/README.md @@ -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= + PG_CONFIG= make + sudo 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= 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 +``` diff --git a/src/test/regress/citus_tests/upgrade/__init__.py b/src/test/regress/citus_tests/upgrade/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/src/test/regress/upgrade/citus_upgrade_test.py b/src/test/regress/citus_tests/upgrade/citus_upgrade_test.py similarity index 51% rename from src/test/regress/upgrade/citus_upgrade_test.py rename to src/test/regress/citus_tests/upgrade/citus_upgrade_test.py index dae2bbaeb..2e2f8f9ea 100755 --- a/src/test/regress/upgrade/citus_upgrade_test.py +++ b/src/test/regress/citus_tests/upgrade/citus_upgrade_test.py @@ -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) diff --git a/src/test/regress/upgrade/generate_citus_tarballs.sh b/src/test/regress/citus_tests/upgrade/generate_citus_tarballs.sh similarity index 100% rename from src/test/regress/upgrade/generate_citus_tarballs.sh rename to src/test/regress/citus_tests/upgrade/generate_citus_tarballs.sh diff --git a/src/test/regress/citus_tests/upgrade/pg_upgrade_test.py b/src/test/regress/citus_tests/upgrade/pg_upgrade_test.py new file mode 100755 index 000000000..f36683cee --- /dev/null +++ b/src/test/regress/citus_tests/upgrade/pg_upgrade_test.py @@ -0,0 +1,137 @@ +#!/usr/bin/env python3 + +"""upgrade_test +Usage: + upgrade_test --old-bindir= --new-bindir= --pgxsdir= + +Options: + --old-bindir= The old PostgreSQL executable directory(ex: '~/.pgenv/pgsql-10.4/bin') + --new-bindir= The new PostgreSQL executable directory(ex: '~/.pgenv/pgsql-11.3/bin') + --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) diff --git a/src/test/regress/upgrade/utils.py b/src/test/regress/citus_tests/utils.py similarity index 75% rename from src/test/regress/upgrade/utils.py rename to src/test/regress/citus_tests/utils.py index 9c1961d8b..8be62c85e 100644 --- a/src/test/regress/upgrade/utils.py +++ b/src/test/regress/citus_tests/utils.py @@ -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): diff --git a/src/test/regress/create_schedule b/src/test/regress/create_schedule new file mode 100644 index 000000000..40a585a2e --- /dev/null +++ b/src/test/regress/create_schedule @@ -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 diff --git a/src/test/regress/expected/ch_benchmarks_1.out b/src/test/regress/expected/ch_benchmarks_1.out new file mode 100644 index 000000000..ec2f81c50 --- /dev/null +++ b/src/test/regress/expected/ch_benchmarks_1.out @@ -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) + diff --git a/src/test/regress/expected/ch_benchmarks_2.out b/src/test/regress/expected/ch_benchmarks_2.out new file mode 100644 index 000000000..9cf69e092 --- /dev/null +++ b/src/test/regress/expected/ch_benchmarks_2.out @@ -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) + diff --git a/src/test/regress/expected/ch_benchmarks_3.out b/src/test/regress/expected/ch_benchmarks_3.out new file mode 100644 index 000000000..aedba5623 --- /dev/null +++ b/src/test/regress/expected/ch_benchmarks_3.out @@ -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) + diff --git a/src/test/regress/expected/ch_benchmarks_4.out b/src/test/regress/expected/ch_benchmarks_4.out new file mode 100644 index 000000000..070607cd6 --- /dev/null +++ b/src/test/regress/expected/ch_benchmarks_4.out @@ -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) + diff --git a/src/test/regress/expected/ch_benchmarks_5.out b/src/test/regress/expected/ch_benchmarks_5.out new file mode 100644 index 000000000..ae8f07a2b --- /dev/null +++ b/src/test/regress/expected/ch_benchmarks_5.out @@ -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) + diff --git a/src/test/regress/expected/ch_benchmarks_6.out b/src/test/regress/expected/ch_benchmarks_6.out new file mode 100644 index 000000000..6c9afd4ab --- /dev/null +++ b/src/test/regress/expected/ch_benchmarks_6.out @@ -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) + diff --git a/src/test/regress/expected/ch_benchmarks_create_load.out b/src/test/regress/expected/ch_benchmarks_create_load.out new file mode 100644 index 000000000..217b53b7b --- /dev/null +++ b/src/test/regress/expected/ch_benchmarks_create_load.out @@ -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'); diff --git a/src/test/regress/expected/create_test.out b/src/test/regress/expected/create_test.out new file mode 100644 index 000000000..c08f7cfb0 --- /dev/null +++ b/src/test/regress/expected/create_test.out @@ -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) + diff --git a/src/test/regress/expected/distributed_planning.out b/src/test/regress/expected/distributed_planning.out new file mode 100644 index 000000000..30101ca96 --- /dev/null +++ b/src/test/regress/expected/distributed_planning.out @@ -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) + diff --git a/src/test/regress/expected/distributed_planning_create_load.out b/src/test/regress/expected/distributed_planning_create_load.out new file mode 100644 index 000000000..19f925bf2 --- /dev/null +++ b/src/test/regress/expected/distributed_planning_create_load.out @@ -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) + diff --git a/src/test/regress/expected/dropped_columns_1.out b/src/test/regress/expected/dropped_columns_1.out new file mode 100644 index 000000000..38b8cec9f --- /dev/null +++ b/src/test/regress/expected/dropped_columns_1.out @@ -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) + diff --git a/src/test/regress/expected/dropped_columns_create_load.out b/src/test/regress/expected/dropped_columns_create_load.out new file mode 100644 index 000000000..1d5bbf4da --- /dev/null +++ b/src/test/regress/expected/dropped_columns_create_load.out @@ -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) + diff --git a/src/test/regress/expected/intermediate_result_pruning_create.out b/src/test/regress/expected/intermediate_result_pruning_create.out new file mode 100644 index 000000000..7d497266d --- /dev/null +++ b/src/test/regress/expected/intermediate_result_pruning_create.out @@ -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); diff --git a/src/test/regress/expected/intermediate_result_pruning_queries_1.out b/src/test/regress/expected/intermediate_result_pruning_queries_1.out new file mode 100644 index 000000000..ed38b565c --- /dev/null +++ b/src/test/regress/expected/intermediate_result_pruning_queries_1.out @@ -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) + diff --git a/src/test/regress/expected/intermediate_result_pruning_queries_2.out b/src/test/regress/expected/intermediate_result_pruning_queries_2.out new file mode 100644 index 000000000..6c7ef3181 --- /dev/null +++ b/src/test/regress/expected/intermediate_result_pruning_queries_2.out @@ -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; diff --git a/src/test/regress/expected/postgres.out b/src/test/regress/expected/postgres.out new file mode 100644 index 000000000..2fdb1714e --- /dev/null +++ b/src/test/regress/expected/postgres.out @@ -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$; diff --git a/src/test/regress/expected/prepared_statements_1.out b/src/test/regress/expected/prepared_statements_1.out new file mode 100644 index 000000000..e71be3f9d --- /dev/null +++ b/src/test/regress/expected/prepared_statements_1.out @@ -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; diff --git a/src/test/regress/expected/prepared_statements_2.out b/src/test/regress/expected/prepared_statements_2.out new file mode 100644 index 000000000..4cd70d873 --- /dev/null +++ b/src/test/regress/expected/prepared_statements_2.out @@ -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) + diff --git a/src/test/regress/expected/prepared_statements_3.out b/src/test/regress/expected/prepared_statements_3.out new file mode 100644 index 000000000..cfec1207a --- /dev/null +++ b/src/test/regress/expected/prepared_statements_3.out @@ -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) + diff --git a/src/test/regress/expected/prepared_statements_4.out b/src/test/regress/expected/prepared_statements_4.out new file mode 100644 index 000000000..0dba296e8 --- /dev/null +++ b/src/test/regress/expected/prepared_statements_4.out @@ -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) + diff --git a/src/test/regress/expected/prepared_statements_create_load.out b/src/test/regress/expected/prepared_statements_create_load.out new file mode 100644 index 000000000..91f653d08 --- /dev/null +++ b/src/test/regress/expected/prepared_statements_create_load.out @@ -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) + diff --git a/src/test/regress/expected/sql_test.out b/src/test/regress/expected/sql_test.out new file mode 100644 index 000000000..377358c9c --- /dev/null +++ b/src/test/regress/expected/sql_test.out @@ -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) + diff --git a/src/test/regress/postgres_schedule b/src/test/regress/postgres_schedule new file mode 100644 index 000000000..b8db7620b --- /dev/null +++ b/src/test/regress/postgres_schedule @@ -0,0 +1 @@ +test: postgres diff --git a/src/test/regress/sql/ch_benchmarks_1.sql b/src/test/regress/sql/ch_benchmarks_1.sql new file mode 100644 index 000000000..479c52c1b --- /dev/null +++ b/src/test/regress/sql/ch_benchmarks_1.sql @@ -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; diff --git a/src/test/regress/sql/ch_benchmarks_2.sql b/src/test/regress/sql/ch_benchmarks_2.sql new file mode 100644 index 000000000..a5ed82453 --- /dev/null +++ b/src/test/regress/sql/ch_benchmarks_2.sql @@ -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; diff --git a/src/test/regress/sql/ch_benchmarks_3.sql b/src/test/regress/sql/ch_benchmarks_3.sql new file mode 100644 index 000000000..e0270e0cd --- /dev/null +++ b/src/test/regress/sql/ch_benchmarks_3.sql @@ -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; diff --git a/src/test/regress/sql/ch_benchmarks_4.sql b/src/test/regress/sql/ch_benchmarks_4.sql new file mode 100644 index 000000000..92ac71f5e --- /dev/null +++ b/src/test/regress/sql/ch_benchmarks_4.sql @@ -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; diff --git a/src/test/regress/sql/ch_benchmarks_5.sql b/src/test/regress/sql/ch_benchmarks_5.sql new file mode 100644 index 000000000..570f1fc8c --- /dev/null +++ b/src/test/regress/sql/ch_benchmarks_5.sql @@ -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); diff --git a/src/test/regress/sql/ch_benchmarks_6.sql b/src/test/regress/sql/ch_benchmarks_6.sql new file mode 100644 index 000000000..a1f35c007 --- /dev/null +++ b/src/test/regress/sql/ch_benchmarks_6.sql @@ -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; diff --git a/src/test/regress/sql/ch_benchmarks_create_load.sql b/src/test/regress/sql/ch_benchmarks_create_load.sql new file mode 100644 index 000000000..07f895ecc --- /dev/null +++ b/src/test/regress/sql/ch_benchmarks_create_load.sql @@ -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'); diff --git a/src/test/regress/sql/create_test.sql b/src/test/regress/sql/create_test.sql new file mode 100644 index 000000000..2f8a5063d --- /dev/null +++ b/src/test/regress/sql/create_test.sql @@ -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'); diff --git a/src/test/regress/sql/distributed_planning.sql b/src/test/regress/sql/distributed_planning.sql new file mode 100644 index 000000000..329a9e9db --- /dev/null +++ b/src/test/regress/sql/distributed_planning.sql @@ -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; diff --git a/src/test/regress/sql/distributed_planning_create_load.sql b/src/test/regress/sql/distributed_planning_create_load.sql new file mode 100644 index 000000000..1ef7abd77 --- /dev/null +++ b/src/test/regress/sql/distributed_planning_create_load.sql @@ -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 *; diff --git a/src/test/regress/sql/dropped_columns_1.sql b/src/test/regress/sql/dropped_columns_1.sql new file mode 100644 index 000000000..4b3f7241f --- /dev/null +++ b/src/test/regress/sql/dropped_columns_1.sql @@ -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'); diff --git a/src/test/regress/sql/dropped_columns_create_load.sql b/src/test/regress/sql/dropped_columns_create_load.sql new file mode 100644 index 000000000..d47c264ea --- /dev/null +++ b/src/test/regress/sql/dropped_columns_create_load.sql @@ -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'); diff --git a/src/test/regress/sql/intermediate_result_pruning_create.sql b/src/test/regress/sql/intermediate_result_pruning_create.sql new file mode 100644 index 000000000..dd06a4986 --- /dev/null +++ b/src/test/regress/sql/intermediate_result_pruning_create.sql @@ -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); diff --git a/src/test/regress/sql/intermediate_result_pruning_queries_1.sql b/src/test/regress/sql/intermediate_result_pruning_queries_1.sql new file mode 100644 index 000000000..b61a8948d --- /dev/null +++ b/src/test/regress/sql/intermediate_result_pruning_queries_1.sql @@ -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; + diff --git a/src/test/regress/sql/intermediate_result_pruning_queries_2.sql b/src/test/regress/sql/intermediate_result_pruning_queries_2.sql new file mode 100644 index 000000000..4193a9af2 --- /dev/null +++ b/src/test/regress/sql/intermediate_result_pruning_queries_2.sql @@ -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; diff --git a/src/test/regress/sql/postgres.sql b/src/test/regress/sql/postgres.sql new file mode 100644 index 000000000..77c6a3a7b --- /dev/null +++ b/src/test/regress/sql/postgres.sql @@ -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$; diff --git a/src/test/regress/sql/prepared_statements_1.sql b/src/test/regress/sql/prepared_statements_1.sql new file mode 100644 index 000000000..ec3f521ad --- /dev/null +++ b/src/test/regress/sql/prepared_statements_1.sql @@ -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; diff --git a/src/test/regress/sql/prepared_statements_2.sql b/src/test/regress/sql/prepared_statements_2.sql new file mode 100644 index 000000000..1f63b4382 --- /dev/null +++ b/src/test/regress/sql/prepared_statements_2.sql @@ -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; + diff --git a/src/test/regress/sql/prepared_statements_3.sql b/src/test/regress/sql/prepared_statements_3.sql new file mode 100644 index 000000000..68ecd40b8 --- /dev/null +++ b/src/test/regress/sql/prepared_statements_3.sql @@ -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; + diff --git a/src/test/regress/sql/prepared_statements_4.sql b/src/test/regress/sql/prepared_statements_4.sql new file mode 100644 index 000000000..e921a83cf --- /dev/null +++ b/src/test/regress/sql/prepared_statements_4.sql @@ -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; + diff --git a/src/test/regress/sql/prepared_statements_create_load.sql b/src/test/regress/sql/prepared_statements_create_load.sql new file mode 100644 index 000000000..b2e5684c0 --- /dev/null +++ b/src/test/regress/sql/prepared_statements_create_load.sql @@ -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'); diff --git a/src/test/regress/sql/sql_test.sql b/src/test/regress/sql/sql_test.sql new file mode 100644 index 000000000..fe12b90c9 --- /dev/null +++ b/src/test/regress/sql/sql_test.sql @@ -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; diff --git a/src/test/regress/sql_base_schedule b/src/test/regress/sql_base_schedule new file mode 100644 index 000000000..9a427bebd --- /dev/null +++ b/src/test/regress/sql_base_schedule @@ -0,0 +1 @@ +test: sql_test diff --git a/src/test/regress/sql_schedule b/src/test/regress/sql_schedule new file mode 100644 index 000000000..7f29ea93b --- /dev/null +++ b/src/test/regress/sql_schedule @@ -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 diff --git a/src/test/regress/upgrade/README.md b/src/test/regress/upgrade/README.md deleted file mode 100644 index 73413949c..000000000 --- a/src/test/regress/upgrade/README.md +++ /dev/null @@ -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= - PG_CONFIG= make - sudo 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= 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 -``` diff --git a/src/test/regress/upgrade/config.py b/src/test/regress/upgrade/config.py deleted file mode 100644 index 3b428054d..000000000 --- a/src/test/regress/upgrade/config.py +++ /dev/null @@ -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, -} diff --git a/src/test/regress/upgrade/pg_upgrade_test.py b/src/test/regress/upgrade/pg_upgrade_test.py deleted file mode 100755 index 076ff6f02..000000000 --- a/src/test/regress/upgrade/pg_upgrade_test.py +++ /dev/null @@ -1,89 +0,0 @@ -#!/usr/bin/env python3 - -"""upgrade_test -Usage: - upgrade_test --old-bindir= --new-bindir= --pgxsdir= - -Options: - --old-bindir= The old PostgreSQL executable directory(ex: '~/.pgenv/pgsql-10.4/bin') - --new-bindir= The new PostgreSQL executable directory(ex: '~/.pgenv/pgsql-11.3/bin') - --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) diff --git a/src/test/regress/upgrade/upgrade_common.py b/src/test/regress/upgrade/upgrade_common.py deleted file mode 100644 index 1ffde7261..000000000 --- a/src/test/regress/upgrade/upgrade_common.py +++ /dev/null @@ -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)