mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #54498 from bakam412/bugfix/local_multiquery_parameter
Bugfix/local multiquery parameter
This commit is contained in:
commit
8d01137d73
@ -394,7 +394,7 @@ do
|
||||
done
|
||||
|
||||
# for each query run, prepare array of metrics from query log
|
||||
clickhouse-local --query "
|
||||
clickhouse-local --multiquery --query "
|
||||
create view query_runs as select * from file('analyze/query-runs.tsv', TSV,
|
||||
'test text, query_index int, query_id text, version UInt8, time float');
|
||||
|
||||
@ -551,7 +551,7 @@ numactl --cpunodebind=all --membind=all numactl --show
|
||||
# If the available memory falls below 2 * size, GNU parallel will suspend some of the running jobs.
|
||||
numactl --cpunodebind=all --membind=all parallel -v --joblog analyze/parallel-log.txt --memsuspend 15G --null < analyze/commands.txt 2>> analyze/errors.log
|
||||
|
||||
clickhouse-local --query "
|
||||
clickhouse-local --multiquery --query "
|
||||
-- Join the metric names back to the metric statistics we've calculated, and make
|
||||
-- a denormalized table of them -- statistics for all metrics for all queries.
|
||||
-- The WITH, ARRAY JOIN and CROSS JOIN do not like each other:
|
||||
@ -649,7 +649,7 @@ rm ./*.{rep,svg} test-times.tsv test-dump.tsv unstable.tsv unstable-query-ids.ts
|
||||
cat analyze/errors.log >> report/errors.log ||:
|
||||
cat profile-errors.log >> report/errors.log ||:
|
||||
|
||||
clickhouse-local --query "
|
||||
clickhouse-local --multiquery --query "
|
||||
create view query_display_names as select * from
|
||||
file('analyze/query-display-names.tsv', TSV,
|
||||
'test text, query_index int, query_display_name text')
|
||||
@ -950,7 +950,7 @@ create table all_query_metrics_tsv engine File(TSV, 'report/all-query-metrics.ts
|
||||
for version in {right,left}
|
||||
do
|
||||
rm -rf data
|
||||
clickhouse-local --query "
|
||||
clickhouse-local --multiquery --query "
|
||||
create view query_profiles as
|
||||
with 0 as left, 1 as right
|
||||
select * from file('analyze/query-profiles.tsv', TSV,
|
||||
@ -1120,7 +1120,7 @@ function report_metrics
|
||||
rm -rf metrics ||:
|
||||
mkdir metrics
|
||||
|
||||
clickhouse-local --query "
|
||||
clickhouse-local --multiquery --query "
|
||||
create view right_async_metric_log as
|
||||
select * from file('right-async-metric-log.tsv', TSVWithNamesAndTypes)
|
||||
;
|
||||
@ -1180,7 +1180,7 @@ function upload_results
|
||||
# Prepare info for the CI checks table.
|
||||
rm -f ci-checks.tsv
|
||||
|
||||
clickhouse-local --query "
|
||||
clickhouse-local --multiquery --query "
|
||||
create view queries as select * from file('report/queries.tsv', TSVWithNamesAndTypes);
|
||||
|
||||
create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv')
|
||||
|
@ -572,17 +572,14 @@ void LocalServer::processConfig()
|
||||
if (!queries.empty() && config().has("queries-file"))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Options '--query' and '--queries-file' cannot be specified at the same time");
|
||||
|
||||
if (config().has("multiquery"))
|
||||
is_multiquery = true;
|
||||
|
||||
delayed_interactive = config().has("interactive") && (!queries.empty() || config().has("queries-file"));
|
||||
if (is_interactive && !delayed_interactive)
|
||||
{
|
||||
if (config().has("multiquery"))
|
||||
is_multiquery = true;
|
||||
}
|
||||
else
|
||||
if (!is_interactive || delayed_interactive)
|
||||
{
|
||||
echo_queries = config().hasOption("echo") || config().hasOption("verbose");
|
||||
ignore_error = config().getBool("ignore-error", false);
|
||||
is_multiquery = true;
|
||||
}
|
||||
|
||||
print_stack_trace = config().getBool("stacktrace", false);
|
||||
|
@ -55,7 +55,7 @@ def test_single_file(started_cluster, cluster):
|
||||
path = get_dist_path(cluster, "distr_1", 1)
|
||||
query = f"select * from file('{path}/1.bin', 'Distributed')"
|
||||
out = node.exec_in_container(
|
||||
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
||||
["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query]
|
||||
)
|
||||
|
||||
assert out == "1\ta\n2\tbb\n3\tccc\n"
|
||||
@ -65,7 +65,7 @@ def test_single_file(started_cluster, cluster):
|
||||
select * from t;
|
||||
"""
|
||||
out = node.exec_in_container(
|
||||
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
||||
["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query]
|
||||
)
|
||||
|
||||
assert out == "1\ta\n2\tbb\n3\tccc\n"
|
||||
@ -106,7 +106,7 @@ def test_two_files(started_cluster, cluster):
|
||||
select * from t order by x;
|
||||
"""
|
||||
out = node.exec_in_container(
|
||||
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
||||
["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query]
|
||||
)
|
||||
|
||||
assert out == "0\t_\n1\ta\n2\tbb\n3\tccc\n"
|
||||
@ -141,7 +141,7 @@ def test_single_file_old(started_cluster, cluster):
|
||||
select * from t;
|
||||
"""
|
||||
out = node.exec_in_container(
|
||||
["/usr/bin/clickhouse", "local", "--stacktrace", "-q", query]
|
||||
["/usr/bin/clickhouse", "local", "--multiquery", "--stacktrace", "-q", query]
|
||||
)
|
||||
|
||||
assert out == "1\ta\n2\tbb\n3\tccc\n"
|
||||
|
@ -48,13 +48,13 @@ pack_unpack_compare "SELECT name, is_aggregate FROM system.functions" "name Stri
|
||||
echo
|
||||
# Check settings are passed correctly
|
||||
${CLICKHOUSE_LOCAL} --max_rows_in_distinct=33 -q "SELECT name, value FROM system.settings WHERE name = 'max_rows_in_distinct'"
|
||||
${CLICKHOUSE_LOCAL} -q "SET max_rows_in_distinct=33; SELECT name, value FROM system.settings WHERE name = 'max_rows_in_distinct'"
|
||||
${CLICKHOUSE_LOCAL} -n -q "SET max_rows_in_distinct=33; SELECT name, value FROM system.settings WHERE name = 'max_rows_in_distinct'"
|
||||
${CLICKHOUSE_LOCAL} --max_bytes_before_external_group_by=1 --max_block_size=10 -q "SELECT sum(ignore(*)) FROM (SELECT number, count() FROM numbers(1000) GROUP BY number)"
|
||||
echo
|
||||
# Check exta options, we expect zero exit code and no stderr output
|
||||
(${CLICKHOUSE_LOCAL} --ignore-error --echo -q "SELECT nothing_to_do();SELECT 42;" 2>/dev/null || echo "Wrong RC")
|
||||
(${CLICKHOUSE_LOCAL} --ignore-error -n --echo -q "SELECT nothing_to_do();SELECT 42;" 2>/dev/null || echo "Wrong RC")
|
||||
echo
|
||||
${CLICKHOUSE_LOCAL} -q "CREATE TABLE sophisticated_default
|
||||
${CLICKHOUSE_LOCAL} -n -q "CREATE TABLE sophisticated_default
|
||||
(
|
||||
a UInt8 DEFAULT 3,
|
||||
b UInt8 ALIAS a + 5,
|
||||
|
@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
echo 'zero rows'
|
||||
for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do
|
||||
echo $format
|
||||
${CLICKHOUSE_LOCAL} --query="
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query="
|
||||
CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1) WHERE number < 0;
|
||||
SELECT * FROM ${format}_01375;
|
||||
DROP TABLE ${format}_01375;
|
||||
@ -22,7 +22,7 @@ echo 'multi clickhouse-local one file'
|
||||
for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do
|
||||
echo $format
|
||||
for _ in {1..2}; do
|
||||
${CLICKHOUSE_LOCAL} --query="
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query="
|
||||
CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1);
|
||||
SELECT * FROM ${format}_01375;
|
||||
DROP TABLE ${format}_01375;
|
||||
|
@ -10,6 +10,6 @@ rm -rf "${WORKING_FOLDER_01527}"
|
||||
mkdir -p "${WORKING_FOLDER_01527}"
|
||||
|
||||
# OPTIMIZE was crashing due to lack of temporary volume in local
|
||||
${CLICKHOUSE_LOCAL} --query "drop database if exists d; create database d; create table d.t engine MergeTree order by a as select 1 a; optimize table d.t final" --path="${WORKING_FOLDER_01527}"
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "drop database if exists d; create database d; create table d.t engine MergeTree order by a as select 1 a; optimize table d.t final" --path="${WORKING_FOLDER_01527}"
|
||||
|
||||
rm -rf "${WORKING_FOLDER_01527}"
|
||||
|
@ -10,7 +10,7 @@ rm -f -- "$the_file"
|
||||
|
||||
# We are going to check that format settings work for File engine,
|
||||
# by creating a table with a non-default delimiter, and reading from it.
|
||||
${CLICKHOUSE_LOCAL} --query "
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "
|
||||
create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|';
|
||||
insert into t select 1 a, 1 b;
|
||||
"
|
||||
@ -18,7 +18,7 @@ ${CLICKHOUSE_LOCAL} --query "
|
||||
# See what's in the file
|
||||
cat "$the_file"
|
||||
|
||||
${CLICKHOUSE_LOCAL} --query "
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "
|
||||
create table t(a int, b int) engine File(CSV, '$the_file') settings format_csv_delimiter = '|';
|
||||
select * from t;
|
||||
"
|
||||
|
@ -15,7 +15,7 @@ mkdir -p "${WORKING_FOLDER_01600}"
|
||||
clickhouse_local() {
|
||||
local query="$1"
|
||||
shift
|
||||
${CLICKHOUSE_LOCAL} --allow_deprecated_database_ordinary=1 --query "$query" "$@" --path="${WORKING_FOLDER_01600}"
|
||||
${CLICKHOUSE_LOCAL} --allow_deprecated_database_ordinary=1 --multiquery --query "$query" "$@" --path="${WORKING_FOLDER_01600}"
|
||||
}
|
||||
|
||||
test_detach_attach_sequence() {
|
||||
|
@ -63,7 +63,7 @@ echo $c_count
|
||||
|
||||
# Valid cases:
|
||||
# The default dir is the CWD path in LOCAL mode
|
||||
${CLICKHOUSE_LOCAL} --query "
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "
|
||||
drop table if exists data;
|
||||
create table data (A String, B String) engine=MergeTree() order by A;
|
||||
select file('a.txt'), file('b.txt');
|
||||
|
@ -11,7 +11,7 @@ cat "$SAMPLE_FILE"
|
||||
|
||||
echo '******************'
|
||||
echo 'Read twice from a regular file'
|
||||
${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table; select * from table;' --file "$SAMPLE_FILE"
|
||||
${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -n -q 'select * from table; select * from table;' --file "$SAMPLE_FILE"
|
||||
echo '---'
|
||||
${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table WHERE x IN (select x from table);' --file "$SAMPLE_FILE"
|
||||
echo '---'
|
||||
@ -19,7 +19,7 @@ ${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table UNI
|
||||
|
||||
echo '******************'
|
||||
echo 'Read twice from file descriptor that corresponds to a regular file'
|
||||
${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table; select * from table;' < "$SAMPLE_FILE"
|
||||
${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -n -q 'select * from table; select * from table;' < "$SAMPLE_FILE"
|
||||
echo '---'
|
||||
${CLICKHOUSE_LOCAL} --structure 'x UInt64, s String' -q 'select * from table WHERE x IN (select x from table);' < "$SAMPLE_FILE"
|
||||
echo '---'
|
||||
|
@ -19,7 +19,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_infile;"
|
||||
|
||||
# if it not fails, select will print information
|
||||
${CLICKHOUSE_LOCAL} --query "CREATE TABLE test_infile (word String) ENGINE=Memory(); INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_TMP}/test_infile.gz' FORMAT CSV; SELECT * from test_infile;"
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "CREATE TABLE test_infile (word String) ENGINE=Memory(); INSERT INTO test_infile FROM INFILE '${CLICKHOUSE_TMP}/test_infile.gz' FORMAT CSV; SELECT * from test_infile;"
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=DROP+TABLE" -d 'IF EXISTS test_infile_url'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=CREATE" -d 'TABLE test_infile_url (x String) ENGINE = Memory'
|
||||
|
@ -55,8 +55,8 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_compression_keyword;"
|
||||
[ -e "${CLICKHOUSE_TMP}"/test_comp_for_input_and_output_without_gz.gz ] && rm "${CLICKHOUSE_TMP}"/test_comp_for_input_and_output_without_gz.gz
|
||||
|
||||
# create files using compression method and without it to check that both queries work correct
|
||||
${CLICKHOUSE_LOCAL} --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated;"
|
||||
${CLICKHOUSE_LOCAL} --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz' COMPRESSION 'GZ' FORMAT TabSeparated;"
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated;"
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "SELECT * FROM (SELECT 'Hello, World! From local.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz' COMPRESSION 'GZ' FORMAT TabSeparated;"
|
||||
|
||||
# check content of files
|
||||
cp ${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_to_decomp.gz
|
||||
@ -68,7 +68,7 @@ gunzip ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz_to_decomp.gz
|
||||
cat ${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz_to_decomp
|
||||
|
||||
# create table to check inserts
|
||||
${CLICKHOUSE_LOCAL} --query "
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "
|
||||
DROP TABLE IF EXISTS test_compression_keyword;
|
||||
CREATE TABLE test_compression_keyword (text String) Engine=Memory;
|
||||
INSERT INTO TABLE test_compression_keyword FROM INFILE '${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz' FORMAT TabSeparated;
|
||||
@ -81,4 +81,4 @@ SELECT * FROM test_compression_keyword;
|
||||
rm -f "${CLICKHOUSE_TMP}/test_comp_for_input_and_output_to_decomp"
|
||||
rm -f "${CLICKHOUSE_TMP}/test_comp_for_input_and_output.gz"
|
||||
rm -f "${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz_to_decomp"
|
||||
rm -f "${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz"
|
||||
rm -f "${CLICKHOUSE_TMP}/test_comp_for_input_and_output_without_gz"
|
||||
|
@ -4,4 +4,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_LOCAL} --query "create table t (n int, m int default 42) engine=Memory;insert into t values (1, NULL);select * from t"
|
||||
${CLICKHOUSE_LOCAL} --multiquery --query "create table t (n int, m int default 42) engine=Memory;insert into t values (1, NULL);select * from t"
|
||||
|
@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
for Engine in Atomic Ordinary; do
|
||||
$CLICKHOUSE_LOCAL --allow_deprecated_database_ordinary=1 --query """
|
||||
$CLICKHOUSE_LOCAL --allow_deprecated_database_ordinary=1 --multiquery --query """
|
||||
CREATE DATABASE foo_$Engine Engine=$Engine;
|
||||
DROP DATABASE foo_$Engine;
|
||||
"""
|
||||
|
@ -1 +1,2 @@
|
||||
OK
|
||||
Missing columns
|
||||
|
@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
|
||||
$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' ||:
|
||||
$CLICKHOUSE_LOCAL --query="SELECT n -- { serverError 47 }"
|
||||
$CLICKHOUSE_LOCAL --query="SELECT n -- { serverError 47 }" 2>&1 | grep -o 'Missing columns'
|
||||
|
@ -12,6 +12,7 @@ grep_options=(
|
||||
-e 'Cannot create file: /no/such/directory'
|
||||
-e 'Cannot load data for command line suggestions:'
|
||||
-e 'ClickHouse local version'
|
||||
-e 'Empty query'
|
||||
)
|
||||
|
||||
ASAN_OPTIONS=$ASAN_OPTIONS:exitcode=3 $CLICKHOUSE_LOCAL --history_file /no/such/directory |& grep -v "${grep_options[@]}"
|
||||
|
@ -6,19 +6,19 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
|
||||
$CLICKHOUSE_LOCAL <<END
|
||||
$CLICKHOUSE_LOCAL --multiquery <<END
|
||||
SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": 1}');
|
||||
SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": null}');
|
||||
END
|
||||
|
||||
$CLICKHOUSE_LOCAL <<END 2>&1 | rg -Fc "'w' character"
|
||||
$CLICKHOUSE_LOCAL --multiquery <<END 2>&1 | rg -Fc "'w' character"
|
||||
SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": wtf}');
|
||||
END
|
||||
|
||||
$CLICKHOUSE_LOCAL <<END 2>&1 | rg -Fc "expected 'false'"
|
||||
$CLICKHOUSE_LOCAL --multiquery <<END 2>&1 | rg -Fc "expected 'false'"
|
||||
SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": ftw}');
|
||||
END
|
||||
|
||||
$CLICKHOUSE_LOCAL <<END 2>&1 | rg -Fc "'{' character"
|
||||
$CLICKHOUSE_LOCAL --multiquery <<END 2>&1 | rg -Fc "'{' character"
|
||||
SELECT * FROM format(JSONEachRow, 'x Bool', '{"x": {}}');
|
||||
END
|
||||
|
@ -10,8 +10,7 @@ Syntax error
|
||||
101
|
||||
202
|
||||
202
|
||||
303
|
||||
303
|
||||
303
|
||||
Multi-statements are not allowed
|
||||
Empty query
|
||||
Bad arguments
|
||||
Syntax error
|
||||
|
@ -15,7 +15,7 @@ $CLICKHOUSE_CLIENT --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax er
|
||||
# clickhouse-local
|
||||
$CLICKHOUSE_LOCAL --query "SELECT 101" --query "SELECT 101"
|
||||
$CLICKHOUSE_LOCAL --query "SELECT 202;" --query "SELECT 202;"
|
||||
$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELECT 303; SELECT 303" 2>&1 # behaves differently than clickhouse-client, TODO make it consistent
|
||||
$CLICKHOUSE_LOCAL --query "" --query "" 2>&1 # behaves equally different than clickhouse-client TODO
|
||||
$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELECT 303; SELECT 303" 2>&1 | grep -o 'Multi-statements are not allowed' # behaves differently than clickhouse-client, TODO make it consistent
|
||||
$CLICKHOUSE_LOCAL --query "" --query "" 2>&1 | grep -o 'Empty query' # behaves equally different than clickhouse-client TODO
|
||||
$CLICKHOUSE_LOCAL --query "SELECT 303" --query 2>&1 | grep -o 'Bad arguments'
|
||||
$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error'
|
||||
|
@ -0,0 +1,6 @@
|
||||
Multi-statements are not allowed
|
||||
Multi-statements are not allowed
|
||||
1
|
||||
2
|
||||
1
|
||||
2
|
14
tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh
Executable file
14
tests/queries/0_stateless/02875_clickhouse_local_multiquery.sh
Executable file
@ -0,0 +1,14 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
# throw exception
|
||||
$CLICKHOUSE_CLIENT -q "select 1; select 2;" 2>&1 | grep -o 'Multi-statements are not allowed'
|
||||
$CLICKHOUSE_LOCAL -q "select 1; select 2;" 2>&1 | grep -o 'Multi-statements are not allowed'
|
||||
# execute correctly
|
||||
$CLICKHOUSE_CLIENT -n -q "select 1; select 2;"
|
||||
$CLICKHOUSE_LOCAL -n -q "select 1; select 2;"
|
||||
|
||||
exit 0
|
Loading…
Reference in New Issue
Block a user