Merge pull request #27928 from Avogar/stress-test

Add backward compatibility check in stress test
This commit is contained in:
Kruglov Pavel 2022-03-21 12:49:46 +01:00 committed by GitHub
commit fbcc27a339
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 301 additions and 20 deletions

View File

@ -25,6 +25,7 @@ RUN apt-get update -y \
brotli
COPY ./stress /stress
COPY ./download_previous_release /download_previous_release
COPY run.sh /
ENV DATASETS="hits visits"

View File

@ -0,0 +1,97 @@
#!/usr/bin/env python3
import requests
import re
import os
CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags"
CLICKHOUSE_COMMON_STATIC_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-common-static_{version}_amd64.deb"
CLICKHOUSE_COMMON_STATIC_DBG_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-common-static-dbg_{version}_amd64.deb"
CLICKHOUSE_SERVER_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-server_{version}_all.deb"
CLICKHOUSE_CLIENT_DOWNLOAD_URL = "https://github.com/ClickHouse/ClickHouse/releases/download/v{version}-{type}/clickhouse-client_{version}_amd64.deb"
CLICKHOUSE_COMMON_STATIC_PACKET_NAME = "clickhouse-common-static_{version}_amd64.deb"
CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME = "clickhouse-common-static-dbg_{version}_amd64.deb"
CLICKHOUSE_SERVER_PACKET_NAME = "clickhouse-server_{version}_all.deb"
CLICKHOUSE_CLIENT_PACKET_NAME = "clickhouse-client_{version}_all.deb"
PACKETS_DIR = "previous_release_package_folder/"
VERSION_PATTERN = r"((?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)"
class Version:
def __init__(self, version):
self.version = version
def __lt__(self, other):
return list(map(int, self.version.split('.'))) < list(map(int, other.version.split('.')))
def __str__(self):
return self.version
class ReleaseInfo:
def __init__(self, version, release_type):
self.version = version
self.type = release_type
def find_previous_release(server_version, releases):
releases.sort(key=lambda x: x.version, reverse=True)
for release in releases:
if release.version < server_version:
return True, release
return False, None
def get_previous_release(server_version):
page = 1
found = False
while not found:
response = requests.get(CLICKHOUSE_TAGS_URL, {'page': page, 'per_page': 100})
if not response.ok:
raise Exception('Cannot load the list of tags from github: ' + response.reason)
releases_str = set(re.findall(VERSION_PATTERN, response.text))
if len(releases_str) == 0:
raise Exception('Cannot find previous release for ' + str(server_version) + ' server version')
releases = list(map(lambda x: ReleaseInfo(Version(x.split('-')[0]), x.split('-')[1]), releases_str))
found, previous_release = find_previous_release(server_version, releases)
page += 1
return previous_release
def download_packet(url, local_file_name):
response = requests.get(url)
print(url)
if response.ok:
open(PACKETS_DIR + local_file_name, 'wb').write(response.content)
def download_packets(release):
if not os.path.exists(PACKETS_DIR):
os.makedirs(PACKETS_DIR)
download_packet(CLICKHOUSE_COMMON_STATIC_DOWNLOAD_URL.format(version=release.version, type=release.type),
CLICKHOUSE_COMMON_STATIC_PACKET_NAME.format(version=release.version))
download_packet(CLICKHOUSE_COMMON_STATIC_DBG_DOWNLOAD_URL.format(version=release.version, type=release.type),
CLICKHOUSE_COMMON_STATIC_DBG_PACKET_NAME.format(version=release.version))
download_packet(CLICKHOUSE_SERVER_DOWNLOAD_URL.format(version=release.version, type=release.type),
CLICKHOUSE_SERVER_PACKET_NAME.format(version=release.version))
download_packet(CLICKHOUSE_CLIENT_DOWNLOAD_URL.format(version=release.version, type=release.type),
CLICKHOUSE_CLIENT_PACKET_NAME.format(version=release.version))
if __name__ == '__main__':
server_version = Version(input())
previous_release = get_previous_release(server_version)
download_packets(previous_release)

View File

@ -22,15 +22,19 @@ export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_PROBABILITY=0.001
export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_PROBABILITY=0.001
export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_PROBABILITY=0.001
export THREAD_FUZZER_pthread_mutex_lock_BEFORE_SLEEP_TIME_US=10000
export THREAD_FUZZER_pthread_mutex_lock_AFTER_SLEEP_TIME_US=10000
export THREAD_FUZZER_pthread_mutex_unlock_BEFORE_SLEEP_TIME_US=10000
export THREAD_FUZZER_pthread_mutex_unlock_AFTER_SLEEP_TIME_US=10000
dpkg -i package_folder/clickhouse-common-static_*.deb
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb
dpkg -i package_folder/clickhouse-server_*.deb
dpkg -i package_folder/clickhouse-client_*.deb
function install_packages()
{
dpkg -i $1/clickhouse-common-static_*.deb
dpkg -i $1/clickhouse-common-static-dbg_*.deb
dpkg -i $1/clickhouse-server_*.deb
dpkg -i $1/clickhouse-client_*.deb
}
function configure()
{
@ -116,7 +120,7 @@ function start()
counter=0
until clickhouse-client --query "SELECT 1"
do
if [ "$counter" -gt 240 ]
if [ "$counter" -gt ${1:-240} ]
then
echo "Cannot start clickhouse-server"
cat /var/log/clickhouse-server/stdout.log
@ -127,6 +131,9 @@ function start()
# use root to match with current uid
clickhouse start --user root >/var/log/clickhouse-server/stdout.log 2>>/var/log/clickhouse-server/stderr.log
sleep 0.5
cat /var/log/clickhouse-server/stdout.log
tail -n200 /var/log/clickhouse-server/stderr.log
tail -n200 /var/log/clickhouse-server/clickhouse-server.log
counter=$((counter + 1))
done
@ -171,6 +178,8 @@ quit
time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||:
}
install_packages package_folder
configure
./setup_minio.sh
@ -246,6 +255,120 @@ zgrep -Fa "########################################" /test_output/* > /dev/null
zgrep -Fa " received signal " /test_output/gdb.log > /dev/null \
&& echo -e 'Found signal in gdb.log\tFAIL' >> /test_output/test_results.tsv
echo -e "Backward compatibility check\n"
echo "Download previous release server"
mkdir previous_release_package_folder
clickhouse-client --query="SELECT version()" | ./download_previous_release && echo -e 'Download script exit code\tOK' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Download script failed\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
if [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ]
then
echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/backward_compatibility_check_results.tsv
stop
# Uninstall current packages
dpkg --remove clickhouse-client
dpkg --remove clickhouse-server
dpkg --remove clickhouse-common-static-dbg
dpkg --remove clickhouse-common-static
rm -rf /var/lib/clickhouse/*
# Install previous release packages
install_packages previous_release_package_folder
# Start server from previous release
configure
start
clickhouse-client --query="SELECT 'Server version: ', version()"
# Install new package before running stress test because we should use new clickhouse-client and new clickhouse-test
install_packages package_folder
mkdir tmp_stress_output
./stress --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \
&& echo -e 'Test script exit code\tOK' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Test script failed\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
rm -rf tmp_stress_output
clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables"
stop
# Start new server
configure
start 500
clickhouse-client --query "SELECT 'Server successfully started', 'OK'" >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Server failed to start\tFAIL' >> /test_output/backward_compatibility_check_results.tsv
clickhouse-client --query="SELECT 'Server version: ', version()"
# Let the server run for a while before checking log.
sleep 60
stop
# Error messages (we should ignore some errors)
zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
-e "Code: 236. DB::Exception: Cancelled mutating parts" \
-e "REPLICA_IS_ALREADY_ACTIVE" \
-e "REPLICA_IS_ALREADY_EXIST" \
-e "DDLWorker: Cannot parse DDL task query" \
-e "RaftInstance: failed to accept a rpc connection due to error 125" \
-e "UNKNOWN_DATABASE" \
-e "NETWORK_ERROR" \
-e "UNKNOWN_TABLE" \
-e "ZooKeeperClient" \
-e "KEEPER_EXCEPTION" \
-e "DirectoryMonitor" \
-e "TABLE_IS_READ_ONLY" \
-e "Code: 1000, e.code() = 111, Connection refused" \
-e "UNFINISHED" \
-e "Renaming unexpected part" \
/var/log/clickhouse-server/clickhouse-server.log | zgrep -Fa "<Error>" > /dev/null \
&& echo -e 'Error message in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No Error messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
# Sanitizer asserts
zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp
zgrep -Fav "ASan doesn't fully support makecontext/swapcontext functions" /test_output/tmp > /dev/null \
&& echo -e 'Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No sanitizer asserts\tOK' >> /test_output/backward_compatibility_check_results.tsv
rm -f /test_output/tmp
# OOM
zgrep -Fa " <Fatal> Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No OOM messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
# Logical errors
zgrep -Fa "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Logical error thrown (see clickhouse-server.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No logical errors\tOK' >> /test_output/backward_compatibility_check_results.tsv
# Crash
zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'Not crashed\tOK' >> /test_output/backward_compatibility_check_results.tsv
# It also checks for crash without stacktrace (printed by watchdog)
zgrep -Fa " <Fatal> " /var/log/clickhouse-server/clickhouse-server.log > /dev/null \
&& echo -e 'Fatal message in clickhouse-server.log\tFAIL' >> /test_output/backward_compatibility_check_results.tsv \
|| echo -e 'No fatal messages in clickhouse-server.log\tOK' >> /test_output/backward_compatibility_check_results.tsv
else
echo -e "Failed to download previous release packets\tFAIL" >> /test_output/backward_compatibility_check_results.tsv
fi
zgrep -Fa "FAIL" /test_output/backward_compatibility_check_results.tsv > /dev/null \
&& echo -e 'Backward compatibility check\tFAIL' >> /test_output/test_results.tsv \
|| echo -e 'Backward compatibility check\tOK' >> /test_output/test_results.tsv
# Put logs into /test_output/
for log_file in /var/log/clickhouse-server/clickhouse-server.log*
do

View File

@ -47,7 +47,8 @@ def get_options(i):
return ' '.join(options)
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit):
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit, backward_compatibility_check):
backward_compatibility_check_option = '--backward-compatibility-check' if backward_compatibility_check else ''
global_time_limit_option = ''
if global_time_limit:
global_time_limit_option = "--global_time_limit={}".format(global_time_limit)
@ -56,7 +57,7 @@ def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_t
pipes = []
for i in range(0, len(output_paths)):
f = open(output_paths[i], 'w')
full_command = "{} {} {} {}".format(cmd, get_options(i), global_time_limit_option, skip_tests_option)
full_command = "{} {} {} {} {}".format(cmd, get_options(i), global_time_limit_option, skip_tests_option, backward_compatibility_check_option)
logging.info("Run func tests '%s'", full_command)
p = Popen(full_command, shell=True, stdout=f, stderr=f)
pipes.append(p)
@ -168,6 +169,7 @@ if __name__ == "__main__":
parser.add_argument("--output-folder")
parser.add_argument("--global-time-limit", type=int, default=1800)
parser.add_argument("--num-parallel", type=int, default=cpu_count())
parser.add_argument('--backward-compatibility-check', action='store_true')
parser.add_argument('--hung-check', action='store_true', default=False)
# make sense only for hung check
parser.add_argument('--drop-databases', action='store_true', default=False)
@ -176,7 +178,7 @@ if __name__ == "__main__":
if args.drop_databases and not args.hung_check:
raise Exception("--drop-databases only used in hung check (--hung-check)")
func_pipes = []
func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests, args.global_time_limit)
func_pipes = run_func_test(args.test_cmd, args.output_folder, args.num_parallel, args.skip_func_tests, args.global_time_limit, args.backward_compatibility_check)
logging.info("Will wait functests to finish")
while True:

View File

@ -820,6 +820,7 @@ void Client::addOptions(OptionsDescription & options_description)
("opentelemetry-tracestate", po::value<std::string>(), "OpenTelemetry tracestate header as described by W3C Trace Context recommendation")
("no-warnings", "disable warnings when client connects to server")
("fake-drop", "Ignore all DROP queries, should be used only for testing")
;
/// Commandline options related to external tables.
@ -952,6 +953,8 @@ void Client::processOptions(const OptionsDescription & options_description,
config().setBool("compression", options["compression"].as<bool>());
if (options.count("no-warnings"))
config().setBool("no-warnings", true);
if (options.count("fake-drop"))
fake_drop = true;
if ((query_fuzzer_runs = options["query-fuzzer-runs"].as<int>()))
{

View File

@ -647,6 +647,12 @@ void ClientBase::processTextAsSingleQuery(const String & full_query)
void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr parsed_query)
{
if (fake_drop)
{
if (parsed_query->as<ASTDropQuery>())
return;
}
/// Rewrite query only when we have query parameters.
/// Note that if query is rewritten, comments in query are lost.
/// But the user often wants to see comments in server logs, query log, processlist, etc.

View File

@ -252,6 +252,8 @@ protected:
QueryProcessingStage::Enum query_processing_stage;
bool fake_drop = false;
struct HostAndPort
{
String host;

View File

@ -59,6 +59,8 @@ MAX_RETRIES = 3
TEST_FILE_EXTENSIONS = ['.sql', '.sql.j2', '.sh', '.py', '.expect']
VERSION_PATTERN = r"^((\d+\.)?(\d+\.)?(\d+\.)?\d+)$"
def stringhash(s):
# default hash() function consistent
@ -354,6 +356,7 @@ class FailureReason(enum.Enum):
REPLICATED_DB = "replicated-database"
S3_STORAGE = "s3-storage"
BUILD = "not running for current build"
BACKWARD_INCOMPATIBLE = "test is backward incompatible"
# UNKNOWN reasons
NO_REFERENCE = "no reference file"
@ -502,6 +505,28 @@ class TestCase:
self.base_url_params = os.environ['CLICKHOUSE_URL_PARAMS'] if 'CLICKHOUSE_URL_PARAMS' in os.environ else ''
self.base_client_options = os.environ['CLICKHOUSE_CLIENT_OPT'] if 'CLICKHOUSE_CLIENT_OPT' in os.environ else ''
# Check if test contains tag "no-backward-compatibility-check" and we should skip it
def check_backward_incompatible_tag(self) -> bool:
for tag in self.tags:
if tag.startswith("no-backward-compatibility-check"):
split = tag.split(':')
# If version is not specified in tag, always skip this test.
if len(split) == 1:
return True
version_from_tag = split[1]
# Check if extracted string from tag is a real ClickHouse version, if not - always skip test.
if re.match(VERSION_PATTERN, version_from_tag) is None:
return True
server_version = str(clickhouse_execute(args, "SELECT version()").decode())
# If server version is less or equal from the version specified in tag, we should skip this test.
if list(map(int, server_version.split('.'))) <= list(map(int, version_from_tag.split('.'))):
return True
return False
# should skip test, should increment skipped_total, skip reason
def should_skip_test(self, suite) -> Optional[FailureReason]:
tags = self.tags
@ -534,10 +559,12 @@ class TestCase:
elif tags and ('no-replicated-database' in tags) and args.replicated_database:
return FailureReason.REPLICATED_DB
elif args.backward_compatibility_check and self.check_backward_incompatible_tag():
return FailureReason.BACKWARD_INCOMPATIBLE
elif tags and ('no-s3-storage' in tags) and args.s3_storage:
return FailureReason.S3_STORAGE
elif tags:
for build_flag in args.build_flags:
if 'no-' + build_flag in tags:
@ -1478,6 +1505,8 @@ if __name__ == '__main__':
group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)')
group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests')
group.add_argument('--backward-compatibility-check', action='store_true', help='Run tests for further backwoard compatibility testing by ignoring all'
'drop queries in tests for collecting data from new version of server')
args = parser.parse_args()
if args.queries and not os.path.isdir(args.queries):
@ -1554,6 +1583,9 @@ if __name__ == '__main__':
else:
args.client_database = 'default'
if args.backward_compatibility_check:
args.client += ' --fake-drop'
if args.client_option:
# Set options for client
if 'CLICKHOUSE_CLIENT_OPT' in os.environ:

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: zookeeper, no-parallel, no-s3-storage
# Tags: zookeeper, no-parallel, no-s3-storage, no-backward-compatibility-check
# Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem
# (it tries to do it as quick as possible, but it still performed in separate thread asynchronously)

View File

@ -1,4 +1,4 @@
-- Tags: long, zookeeper, no-replicated-database
-- Tags: long, zookeeper, no-replicated-database, no-backward-compatibility-check
-- Tag no-replicated-database: Fails due to additional replicas or shards
SET send_logs_level = 'fatal';

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-backward-compatibility-check
# Test fix for issue #5066
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: race, zookeeper, no-parallel
# Tags: race, zookeeper, no-parallel, no-backward-compatibility-check
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: zookeeper, no-parallel, no-fasttest
# Tags: zookeeper, no-parallel, no-fasttest, no-backward-compatibility-check
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: no-backward-compatibility-check:21.12.1.8761
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -1,4 +1,4 @@
-- Tags: no-parallel
-- Tags: no-parallel, no-backward-compatibility-check
DROP DATABASE IF EXISTS test_01191;
CREATE DATABASE test_01191 ENGINE=Atomic;

View File

@ -1,3 +1,5 @@
-- Tags: no-backward-compatibility-check
-- force data path with the user/pass in it
set use_compact_format_in_distributed_parts_names=0;
-- use async send even for localhost

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: no-backward-compatibility-check
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -369,4 +369,4 @@ GROUP BY
user_name, interface, type
ORDER BY
user_name, interface, type;
EOF
EOF

View File

@ -1,3 +1,5 @@
-- Tags: no-backward-compatibility-check
SET mutations_sync = 2;
DROP TABLE IF EXISTS t_sparse_alter;

View File

@ -31,6 +31,8 @@ select * from test_wide_nested;
alter table test_wide_nested update `info.id` = [100,200], `info.age` = [10,20,30], `info.name` = ['a','b','c'] where id = 0; -- { serverError 341 }
kill mutation where table = 'test_wide_nested' and database = currentDatabase() format Null;
-- Recreate table, because KILL MUTATION is not suitable for parallel tests execution.
SELECT '********* test 2 **********';
DROP TABLE test_wide_nested;
@ -54,6 +56,8 @@ select * from test_wide_nested;
alter table test_wide_nested update `info.id` = [100,200,300], `info.age` = [10,20,30] where id = 1; -- { serverError 341 }
kill mutation where table = 'test_wide_nested' and database = currentDatabase() format Null;
DROP TABLE test_wide_nested;
SELECT '********* test 3 **********';

View File

@ -19,7 +19,6 @@ DB_PATH2=$CUR_DIR/${CURR_DATABASE}_db2
function cleanup()
{
${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS ${CURR_DATABASE}"
rm -r "${DB_PATH}" "${DB_PATH2}"
}
trap cleanup EXIT

View File

@ -1,3 +1,5 @@
-- Tags: no-backward-compatibility-check:21.9.1.1
CREATE TABLE IF NOT EXISTS sample_incorrect
(`x` UUID)
ENGINE = MergeTree

View File

@ -1,3 +1,5 @@
-- Tags: no-backward-compatibility-check:21.9.1.1
CREATE TABLE a (number UInt64) ENGINE = MergeTree ORDER BY if(now() > toDateTime('2020-06-01 13:31:40'), toInt64(number), -number); -- { serverError 36 }
CREATE TABLE b (number UInt64) ENGINE = MergeTree ORDER BY now() > toDateTime(number); -- { serverError 36 }
CREATE TABLE c (number UInt64) ENGINE = MergeTree ORDER BY now(); -- { serverError 36 }

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: no-backward-compatibility-check
set -eu

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: no-backward-compatibility-check
set -eu

View File

@ -26,8 +26,8 @@ SOURCE(CLICKHOUSE(TABLE 'test_table'));
CREATE TABLE test_table_default
(
data_1 DEFAULT dictGetUInt64('test_dictionary', 'data_column_1', toUInt64(0)),
data_2 DEFAULT dictGet(test_dictionary, 'data_column_2', toUInt64(0))
data_1 DEFAULT dictGetUInt64('02097_db.test_dictionary', 'data_column_1', toUInt64(0)),
data_2 DEFAULT dictGet(02097_db.test_dictionary, 'data_column_2', toUInt64(0))
)
ENGINE=TinyLog;

View File

@ -1,5 +1,5 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-parallel
# Tags: no-fasttest, no-parallel, no-backward-compatibility-check
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh