Merge pull request #42197 from ClickHouse/add-test-658

Add a test for #658
This commit is contained in:
Alexey Milovidov 2022-10-21 21:56:51 +02:00 committed by GitHub
commit f64a48874a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 4 additions and 214 deletions

View File

@ -1,10 +1,8 @@
#if defined(OS_LINUX)
# include <sys/syscall.h>
#endif
#include <cstdlib>
#include <unistd.h>
#include <base/safeExit.h>
#include <base/defines.h>
[[noreturn]] void safeExit(int code)
{

View File

@ -0,0 +1 @@
1803

View File

@ -0,0 +1,3 @@
-- Even in presense of OR, we evaluate the "0 IN (1, 2, 3)" as a constant expression therefore it does not prevent the index analysis.
SELECT count() FROM test.hits WHERE CounterID IN (14917930, 33034174) OR 0 IN (1, 2, 3) SETTINGS max_rows_to_read = 1000000, force_primary_key = 1;

View File

@ -1,110 +0,0 @@
Testing 00001_count_hits.sql ----> Ok! ✅
Testing 00002_count_visits.sql ----> Ok! ✅
Testing 00004_top_counters.sql ----> Ok! ✅
Testing 00005_filtering.sql ----> Ok! ✅
Testing 00006_agregates.sql ----> Ok! ✅
Testing 00007_uniq.sql ----> Ok! ✅
Testing 00008_uniq.sql ----> Ok! ✅
Testing 00009_uniq_distributed.sql ----> Ok! ✅
Testing 00010_quantiles_segfault.sql ----> Ok! ✅
Testing 00011_sorting.sql ----> Ok! ✅
Testing 00012_sorting_distributed.sql ----> Ok! ✅
Skipping 00013_sorting_of_nested.sql
Testing 00014_filtering_arrays.sql ----> Ok! ✅
Testing 00015_totals_and_no_aggregate_functions.sql ----> Ok! ✅
Testing 00016_any_if_distributed_cond_always_false.sql ----> Ok! ✅
Testing 00017_aggregation_uninitialized_memory.sql ----> Ok! ✅
Testing 00020_distinct_order_by_distributed.sql ----> Ok! ✅
Testing 00021_1_select_with_in.sql ----> Ok! ✅
Testing 00021_2_select_with_in.sql ----> Ok! ✅
Testing 00021_3_select_with_in.sql ----> Ok! ✅
Testing 00022_merge_prewhere.sql ----> Ok! ✅
Testing 00023_totals_limit.sql ----> Ok! ✅
Testing 00024_random_counters.sql ----> Ok! ✅
Testing 00030_array_enumerate_uniq.sql ----> Ok! ✅
Testing 00031_array_enumerate_uniq.sql ----> Ok! ✅
Testing 00032_aggregate_key64.sql ----> Ok! ✅
Testing 00033_aggregate_key_string.sql ----> Ok! ✅
Testing 00034_aggregate_key_fixed_string.sql ----> Ok! ✅
Testing 00035_aggregate_keys128.sql ----> Ok! ✅
Testing 00036_aggregate_hashed.sql ----> Ok! ✅
Testing 00037_uniq_state_merge1.sql ----> Ok! ✅
Testing 00038_uniq_state_merge2.sql ----> Ok! ✅
Testing 00039_primary_key.sql ----> Ok! ✅
Testing 00040_aggregating_materialized_view.sql ----> Ok! ✅
Testing 00041_aggregating_materialized_view.sql ----> Ok! ✅
Testing 00042_any_left_join.sql ----> Ok! ✅
Testing 00043_any_left_join.sql ----> Ok! ✅
Testing 00044_any_left_join_string.sql ----> Ok! ✅
Testing 00045_uniq_upto.sql ----> Ok! ✅
Testing 00046_uniq_upto_distributed.sql ----> Ok! ✅
Testing 00047_bar.sql ----> Ok! ✅
Testing 00048_min_max.sql ----> Ok! ✅
Testing 00049_max_string_if.sql ----> Ok! ✅
Testing 00050_min_max.sql ----> Ok! ✅
Testing 00051_min_max_array.sql ----> Ok! ✅
Testing 00052_group_by_in.sql ----> Ok! ✅
Testing 00053_replicate_segfault.sql ----> Ok! ✅
Testing 00054_merge_tree_partitions.sql ----> Ok! ✅
Testing 00055_index_and_not.sql ----> Ok! ✅
Testing 00056_view.sql ----> Ok! ✅
Testing 00059_merge_sorting_empty_array_joined.sql ----> Ok! ✅
Testing 00060_move_to_prewhere_and_sets.sql ----> Ok! ✅
Skipping 00061_storage_buffer.sql
Testing 00062_loyalty.sql ----> Ok! ✅
Testing 00063_loyalty_joins.sql ----> Ok! ✅
Testing 00065_loyalty_with_storage_join.sql ----> Ok! ✅
Testing 00066_sorting_distributed_many_replicas.sql ----> Ok! ✅
Testing 00067_union_all.sql ----> Ok! ✅
Testing 00068_subquery_in_prewhere.sql ----> Ok! ✅
Testing 00069_duplicate_aggregation_keys.sql ----> Ok! ✅
Testing 00071_merge_tree_optimize_aio.sql ----> Ok! ✅
Testing 00072_compare_date_and_string_index.sql ----> Ok! ✅
Testing 00073_uniq_array.sql ----> Ok! ✅
Testing 00074_full_join.sql ----> Ok! ✅
Testing 00075_left_array_join.sql ----> Ok! ✅
Testing 00076_system_columns_bytes.sql ----> Ok! ✅
Testing 00077_log_tinylog_stripelog.sql ----> Ok! ✅
Testing 00078_group_by_arrays.sql ----> Ok! ✅
Testing 00079_array_join_not_used_joined_column.sql ----> Ok! ✅
Testing 00080_array_join_and_union.sql ----> Ok! ✅
Testing 00081_group_by_without_key_and_totals.sql ----> Ok! ✅
Testing 00082_quantiles.sql ----> Ok! ✅
Testing 00083_array_filter.sql ----> Ok! ✅
Testing 00084_external_aggregation.sql ----> Ok! ✅
Testing 00085_monotonic_evaluation_segfault.sql ----> Ok! ✅
Testing 00086_array_reduce.sql ----> Ok! ✅
Testing 00087_where_0.sql ----> Ok! ✅
Testing 00088_global_in_one_shard_and_rows_before_limit.sql ----> Ok! ✅
Testing 00089_position_functions_with_non_constant_arg.sql ----> Ok! ✅
Testing 00091_prewhere_two_conditions.sql ----> Ok! ✅
Testing 00093_prewhere_array_join.sql ----> Ok! ✅
Testing 00094_order_by_array_join_limit.sql ----> Ok! ✅
Skipping 00095_hyperscan_profiler.sql
Testing 00139_like.sql ----> Ok! ✅
Skipping 00140_rename.sql
Testing 00141_transform.sql ----> Ok! ✅
Testing 00142_system_columns.sql ----> Ok! ✅
Testing 00143_transform_non_const_default.sql ----> Ok! ✅
Testing 00144_functions_of_aggregation_states.sql ----> Ok! ✅
Testing 00145_aggregate_functions_statistics.sql ----> Ok! ✅
Testing 00146_aggregate_function_uniq.sql ----> Ok! ✅
Testing 00147_global_in_aggregate_function.sql ----> Ok! ✅
Testing 00148_monotonic_functions_and_index.sql ----> Ok! ✅
Testing 00149_quantiles_timing_distributed.sql ----> Ok! ✅
Testing 00150_quantiles_timing_precision.sql ----> Ok! ✅
Testing 00151_order_by_read_in_order.sql ----> Ok! ✅
Skipping 00151_replace_partition_with_different_granularity.sql
Skipping 00152_insert_different_granularity.sql
Testing 00153_aggregate_arena_race.sql ----> Ok! ✅
Skipping 00154_avro.sql
Testing 00156_max_execution_speed_sample_merge.sql ----> Ok! ✅
Skipping 00157_cache_dictionary.sql
Skipping 00158_cache_dictionary_has.sql
Testing 00160_decode_xml_component.sql ----> Ok! ✅
Testing 00162_mmap_compression_none.sql ----> Ok! ✅
Testing 00164_quantileBfloat16.sql ----> Ok! ✅
Testing 00165_jit_aggregate_functions.sql ----> Ok! ✅
Skipping 00166_explain_estimate.sql
Testing 00167_read_bytes_from_fs.sql ----> Ok! ✅
Total failed tests:

View File

@ -1,102 +0,0 @@
#!/usr/bin/env bash
# Tags: no-tsan, no-random-settings
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# set -e
# All replicas are localhost, disable `prefer_localhost_replica` option to test network interface
# Currently this feature could not work with hedged requests
# Enabling `enable_sample_offset_parallel_processing` feature could lead to intersecting marks, so some of them would be thrown away and it will lead to incorrect result of SELECT query
SETTINGS="--max_parallel_replicas=3 --use_hedged_requests=false --allow_experimental_parallel_reading_from_replicas=true"
# Prepare tables
$CLICKHOUSE_CLIENT $SETTINGS -nm -q '''
drop table if exists test.dist_hits SYNC;
drop table if exists test.dist_visits SYNC;
create table test.dist_hits as test.hits engine = Distributed("test_cluster_one_shard_three_replicas_localhost", test, hits, rand());
create table test.dist_visits as test.visits engine = Distributed("test_cluster_one_shard_three_replicas_localhost", test, visits, rand());
''';
FAILED=()
# PreviouslyFailed=(
# )
SkipList=(
"00013_sorting_of_nested.sql" # It contains FINAL, which is not allowed together with parallel reading
"00061_storage_buffer.sql"
"00095_hyperscan_profiler.sql" # too long in debug (there is a --no-debug tag inside a test)
"00140_rename.sql" # Multiple renames are not allowed with DatabaseReplicated and tags are not forwarded through this test
"00154_avro.sql" # Plain select * with limit with Distributed table is not deterministic
"00151_replace_partition_with_different_granularity.sql" # Replace partition from Distributed is not allowed
"00152_insert_different_granularity.sql" # The same as above
"00157_cache_dictionary.sql" # Too long in debug mode, but result is correct
"00158_cache_dictionary_has.sql" # The same as above
"00166_explain_estimate.sql" # Distributed table returns nothing
)
# for TESTPATH in "${PreviouslyFailed[@]}"
for TESTPATH in "$CURDIR"/*.sql;
do
TESTNAME=$(basename $TESTPATH)
NUM=$(echo "${TESTNAME}" | grep -o -P '^\d+' | sed 's/^0*//')
if [[ "${NUM}" -ge 168 ]]; then
continue
fi
if [[ " ${SkipList[*]} " =~ ${TESTNAME} ]]; then
echo "Skipping $TESTNAME "
continue
fi
echo -n "Testing $TESTNAME ----> "
# prepare test
NEW_TESTNAME="/tmp/dist_$TESTNAME"
# Added g to sed command to replace all tables, not the first
cat $TESTPATH | sed -e 's/test.hits/test.dist_hits/g' | sed -e 's/test.visits/test.dist_visits/g' > $NEW_TESTNAME
TESTNAME_RESULT="/tmp/result_$TESTNAME"
NEW_TESTNAME_RESULT="/tmp/result_dist_$TESTNAME"
$CLICKHOUSE_CLIENT $SETTINGS -nm < $TESTPATH > $TESTNAME_RESULT
$CLICKHOUSE_CLIENT $SETTINGS -nm < $NEW_TESTNAME > $NEW_TESTNAME_RESULT
expected=$(cat $TESTNAME_RESULT | md5sum)
actual=$(cat $NEW_TESTNAME_RESULT | md5sum)
if [[ "$expected" != "$actual" ]]; then
FAILED+=("$TESTNAME")
echo "Failed! ❌"
echo "Plain:"
cat $TESTNAME_RESULT
echo "Distributed:"
cat $NEW_TESTNAME_RESULT
else
echo "Ok! ✅"
fi
done
echo "Total failed tests: "
# Iterate the loop to read and print each array element
for value in "${FAILED[@]}"
do
echo "🔺 $value"
done
# Drop tables
$CLICKHOUSE_CLIENT $SETTINGS -nm -q '''
drop table if exists test.dist_hits SYNC;
drop table if exists test.dist_visits SYNC;
''';