Merge branch 'master' into format-settings-parsing

This commit is contained in:
Alexey Milovidov 2024-07-13 22:21:20 +02:00
commit d20c0c6593
14 changed files with 311 additions and 211 deletions

View File

@ -36,6 +36,7 @@ disable = '''
bare-except,
no-else-return,
global-statement,
f-string-without-interpolation,
'''
[tool.pylint.SIMILARITIES]

View File

@ -101,9 +101,8 @@ struct MergeTreePartInfo
bool isFakeDropRangePart() const
{
/// Another max level was previously used for REPLACE/MOVE PARTITION
auto another_max_level = std::numeric_limits<decltype(level)>::max();
return level == MergeTreePartInfo::MAX_LEVEL || level == another_max_level;
/// LEGACY_MAX_LEVEL was previously used for REPLACE/MOVE PARTITION
return level == MergeTreePartInfo::MAX_LEVEL || level == MergeTreePartInfo::LEGACY_MAX_LEVEL;
}
String getPartNameAndCheckFormat(MergeTreeDataFormatVersion format_version) const;

View File

@ -325,8 +325,8 @@ def _mark_success_action(
# do nothing, exit without failure
print(f"ERROR: no status file for job [{job}]")
if job_config.run_always or job_config.run_by_label:
print(f"Job [{job}] runs always or by label in CI - do not cache")
if job_config.run_by_label or not job_config.has_digest():
print(f"Job [{job}] has no digest or run by label in CI - do not cache")
else:
if pr_info.is_master:
pass
@ -1125,6 +1125,7 @@ def main() -> int:
### POST action: start
elif args.post:
has_oom_error = False
if Shell.check(
"sudo dmesg -T | grep -q -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e 'oom-kill:constraint=CONSTRAINT_NONE'"
):
@ -1132,6 +1133,7 @@ def main() -> int:
CIBuddy(dry_run=not pr_info.is_release).post_error(
"Out Of Memory", job_name=_get_ext_check_name(args.job_name)
)
has_oom_error = True
job_report = JobReport.load() if JobReport.exist() else None
if job_report:
@ -1235,8 +1237,25 @@ def main() -> int:
ch_helper,
)
else:
# no job report
print(f"No job report for {[args.job_name]} - do nothing")
if CI.is_test_job(args.job_name):
if has_oom_error:
description = "ERROR: Out Of Memory"
else:
description = "ERROR: Unknown job status"
gh = GitHub(get_best_robot_token(), per_page=100)
commit = get_commit(gh, pr_info.sha)
post_commit_status(
commit,
ERROR,
"",
description,
job_report.check_name or _get_ext_check_name(args.job_name),
pr_info,
dump_to_file=True,
)
else:
# no job report
print(f"No job report for {[args.job_name]} - do nothing")
### POST action: end
### MARK SUCCESS action: start

View File

@ -609,7 +609,7 @@ class CiCache:
pushes pending records for all jobs that supposed to be run
"""
for job, job_config in self.jobs_to_do.items():
if job_config.run_always:
if not job_config.has_digest():
continue
pending_state = PendingState(time.time(), run_url=GITHUB_RUN_URL)
assert job_config.batches
@ -680,7 +680,7 @@ class CiCache:
It removes jobs from @jobs_to_do if it is a:
1. test job and it is in @jobs_to_wait (no need to wait not affected jobs in PRs)
2. test job and it has finished on release branch (even if failed)
2. build job which is not required by any test job that is left in @jobs_to_do
3. build job which is not required by any test job that is left in @jobs_to_do
:return:
"""

View File

@ -13,6 +13,9 @@ class CI:
each config item in the below dicts should be an instance of JobConfig class or inherited from it
"""
MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI = 5
MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI = 2
# reimport types to CI class so that they visible as CI.* and mypy is happy
# pylint:disable=useless-import-alias,reimported,import-outside-toplevel
from ci_definitions import BuildConfig as BuildConfig

View File

@ -327,6 +327,9 @@ class JobConfig:
assert self.required_builds
return self.required_builds[0]
def has_digest(self) -> bool:
return self.digest != DigestConfig()
class CommonJobConfigs:
"""
@ -440,7 +443,12 @@ class CommonJobConfigs:
)
ASTFUZZER_TEST = JobConfig(
job_name_keyword="ast",
digest=DigestConfig(),
digest=DigestConfig(
include_paths=[
"./tests/ci/ast_fuzzer_check.py",
],
docker=["clickhouse/fuzzer"],
),
run_command="ast_fuzzer_check.py",
run_always=True,
runner_type=Runners.FUZZER_UNIT_TESTER,

View File

@ -1,8 +1,9 @@
import os
import re
import subprocess
from contextlib import contextmanager
from pathlib import Path
from typing import Any, Iterator, List, Union
from typing import Any, Iterator, List, Union, Optional
class WithIter(type):
@ -83,3 +84,15 @@ class Shell:
check=False,
)
return result.returncode == 0
class Utils:
@staticmethod
def get_failed_tests_number(description: str) -> Optional[int]:
description = description.lower()
pattern = r"fail:\s*(\d+)\s*(?=,|$)"
match = re.search(pattern, description)
if match:
return int(match.group(1))
return None

View File

@ -26,6 +26,8 @@ from pr_info import PRInfo
from report import SUCCESS, FAILURE
from env_helper import GITHUB_UPSTREAM_REPOSITORY, GITHUB_REPOSITORY
from synchronizer_utils import SYNC_BRANCH_PREFIX
from ci_config import CI
from ci_utils import Utils
# The team name for accepted approvals
TEAM_NAME = getenv("GITHUB_TEAM_NAME", "core")
@ -251,23 +253,77 @@ def main():
# set mergeable check status and exit
commit = get_commit(gh, args.pr_info.sha)
statuses = get_commit_filtered_statuses(commit)
state = trigger_mergeable_check(
commit,
statuses,
workflow_failed=(args.wf_status != "success"),
)
# Process upstream StatusNames.SYNC
pr_info = PRInfo()
if (
pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/")
and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY
):
print("Updating upstream statuses")
update_upstream_sync_status(pr_info, state)
max_failed_tests_per_job = 0
job_name_with_max_failures = None
total_failed_tests = 0
failed_to_get_info = False
has_failed_statuses = False
for status in statuses:
if not CI.is_required(status.context):
continue
if status.state == FAILURE:
has_failed_statuses = True
failed_cnt = Utils.get_failed_tests_number(status.description)
if failed_cnt is None:
failed_to_get_info = True
else:
if failed_cnt > max_failed_tests_per_job:
job_name_with_max_failures = status.context
max_failed_tests_per_job = failed_cnt
total_failed_tests += failed_cnt
elif status.state != SUCCESS and status.context not in (
CI.StatusNames.SYNC,
CI.StatusNames.PR_CHECK,
):
# do not block CI on failures in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK)
has_failed_statuses = True
print(
f"Unexpected status for [{status.context}]: [{status.state}] - block further testing"
)
failed_to_get_info = True
if args.wf_status != "success":
# exit with 1 to rerun on workflow failed job restart
can_continue = True
if total_failed_tests > CI.MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI:
print(
f"Required check has [{total_failed_tests}] failed - block further testing"
)
can_continue = False
if max_failed_tests_per_job > CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI:
print(
f"Job [{job_name_with_max_failures}] has [{max_failed_tests_per_job}] failures - block further testing"
)
can_continue = False
if failed_to_get_info:
print("Unexpected commit status state - block further testing")
can_continue = False
if args.wf_status != SUCCESS and not has_failed_statuses:
# workflow failed but reason is unknown as no failed statuses present
can_continue = False
print(
"WARNING: Either the runner is faulty or the operating status is unknown. The first is self-healing, the second requires investigation."
)
if args.wf_status == SUCCESS or has_failed_statuses:
# do not set mergeable check status if args.wf_status == failure, apparently it has died runners and is to be restarted
state = trigger_mergeable_check(
commit,
statuses,
)
# Process upstream StatusNames.SYNC
pr_info = PRInfo()
if (
pr_info.head_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/")
and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY
):
print("Updating upstream statuses")
update_upstream_sync_status(pr_info, state)
else:
print(
"Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status"
)
if not can_continue:
sys.exit(1)
sys.exit(0)

View File

@ -711,7 +711,7 @@ def get_localzone():
class SettingsRandomizer:
settings = {
"max_insert_threads": lambda: 32
"max_insert_threads": lambda: 12
if random.random() < 0.03
else random.randint(1, 3),
"group_by_two_level_threshold": threshold_generator(0.2, 0.2, 1, 1000000),
@ -729,7 +729,7 @@ class SettingsRandomizer:
"prefer_localhost_replica": lambda: random.randint(0, 1),
"max_block_size": lambda: random.randint(8000, 100000),
"max_joined_block_size_rows": lambda: random.randint(8000, 100000),
"max_threads": lambda: 64 if random.random() < 0.03 else random.randint(1, 3),
"max_threads": lambda: 32 if random.random() < 0.03 else random.randint(1, 3),
"optimize_append_index": lambda: random.randint(0, 1),
"optimize_if_chain_to_multiif": lambda: random.randint(0, 1),
"optimize_if_transform_strings_to_enum": lambda: random.randint(0, 1),

View File

@ -5,20 +5,23 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
TMP_DIR="/tmp"
TMP_DIR=${CLICKHOUSE_TMP}/tmp
DATA_DIR=${CLICKHOUSE_TMP}/data
mkdir -p $TMP_DIR
mkdir -p $DATA_DIR
declare -a SearchTypes=("POLYGON" "POLYGON_SIMPLE" "POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL")
tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${CURDIR}"
tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${DATA_DIR}"
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE IF EXISTS points;
CREATE TABLE points (x Float64, y Float64) ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --max_insert_block_size=100000 < "${CURDIR}/01037_point_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --max_insert_block_size=100000 < "${DATA_DIR}/01037_point_data"
rm "${CURDIR}"/01037_point_data
rm "${DATA_DIR}"/01037_point_data
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE IF EXISTS polygons_array;
@ -32,9 +35,9 @@ CREATE TABLE polygons_array
ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${DATA_DIR}/01037_polygon_data"
rm "${CURDIR}"/01037_polygon_data
rm "${DATA_DIR}"/01037_polygon_data
for type in "${SearchTypes[@]}";
do

View File

@ -5,19 +5,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
TMP_DIR="/tmp"
TMP_DIR=${CLICKHOUSE_TMP}/tmp
DATA_DIR=${CLICKHOUSE_TMP}/data
mkdir -p $TMP_DIR
mkdir -p $DATA_DIR
declare -a SearchTypes=("POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL")
tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${CURDIR}"
tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${DATA_DIR}"
$CLICKHOUSE_CLIENT -n --query="
CREATE TABLE points (x Float64, y Float64) ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_point_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${DATA_DIR}/01037_point_data"
rm "${CURDIR}"/01037_point_data
rm "${DATA_DIR}"/01037_point_data
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE IF EXISTS polygons_array;
@ -31,9 +34,9 @@ CREATE TABLE polygons_array
ENGINE = Memory;
"
$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data"
$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${DATA_DIR}/01037_polygon_data"
rm "${CURDIR}"/01037_polygon_data
rm "${DATA_DIR}"/01037_polygon_data
for type in "${SearchTypes[@]}";
do

View File

@ -1,104 +1,104 @@
dictGet test_01037.dict_array (-100,-42) qqq 101
dictGet test_01037.dict_array (-1,0) Click South 423
dictGet test_01037.dict_array (-0.1,0) Click South 423
dictGet test_01037.dict_array (0,-2) Click West 424
dictGet test_01037.dict_array (0,-1.1) Click West 424
dictGet test_01037.dict_array (0,1.1) Click North 422
dictGet test_01037.dict_array (0,2) Click North 422
dictGet test_01037.dict_array (0.1,0) Click East 421
dictGet test_01037.dict_array (0.99,2.99) Click North 422
dictGet test_01037.dict_array (1,0) Click East 421
dictGet test_01037.dict_array (3,3) House 314159
dictGet test_01037.dict_array (5,6) Click 42
dictGet test_01037.dict_array (7.01,7.01) qqq 101
dictGetOrDefault test_01037.dict_array (-100,-42) www 1234
dictGetOrDefault test_01037.dict_array (-1,0) Click South 423
dictGetOrDefault test_01037.dict_array (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_array (0,-2) Click West 424
dictGetOrDefault test_01037.dict_array (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_array (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_array (0,2) Click North 422
dictGetOrDefault test_01037.dict_array (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_array (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_array (1,0) Click East 421
dictGetOrDefault test_01037.dict_array (3,3) House 314159
dictGetOrDefault test_01037.dict_array (5,6) Click 42
dictGetOrDefault test_01037.dict_array (7.01,7.01) www 1234
dictGetOrDefault test_01037.dict_array (-100,-42) dd 44
dictGetOrDefault test_01037.dict_array (-1,0) Click South 423
dictGetOrDefault test_01037.dict_array (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_array (0,-2) Click West 424
dictGetOrDefault test_01037.dict_array (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_array (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_array (0,2) Click North 422
dictGetOrDefault test_01037.dict_array (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_array (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_array (1,0) Click East 421
dictGetOrDefault test_01037.dict_array (3,3) House 314159
dictGetOrDefault test_01037.dict_array (5,6) Click 42
dictGetOrDefault test_01037.dict_array (7.01,7.01) ee 55
dictGet test_01037.dict_tuple (-100,-42) qqq 101
dictGet test_01037.dict_tuple (-1,0) Click South 423
dictGet test_01037.dict_tuple (-0.1,0) Click South 423
dictGet test_01037.dict_tuple (0,-2) Click West 424
dictGet test_01037.dict_tuple (0,-1.1) Click West 424
dictGet test_01037.dict_tuple (0,1.1) Click North 422
dictGet test_01037.dict_tuple (0,2) Click North 422
dictGet test_01037.dict_tuple (0.1,0) Click East 421
dictGet test_01037.dict_tuple (0.99,2.99) Click North 422
dictGet test_01037.dict_tuple (1,0) Click East 421
dictGet test_01037.dict_tuple (3,3) House 314159
dictGet test_01037.dict_tuple (5,6) Click 42
dictGet test_01037.dict_tuple (7.01,7.01) qqq 101
dictGetOrDefault test_01037.dict_tuple (-100,-42) www 1234
dictGetOrDefault test_01037.dict_tuple (-1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (0,-2) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_tuple (0,2) Click North 422
dictGetOrDefault test_01037.dict_tuple (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_tuple (1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (3,3) House 314159
dictGetOrDefault test_01037.dict_tuple (5,6) Click 42
dictGetOrDefault test_01037.dict_tuple (7.01,7.01) www 1234
dictGetOrDefault test_01037.dict_tuple (-100,-42) dd 44
dictGetOrDefault test_01037.dict_tuple (-1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (-0.1,0) Click South 423
dictGetOrDefault test_01037.dict_tuple (0,-2) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,-1.1) Click West 424
dictGetOrDefault test_01037.dict_tuple (0,1.1) Click North 422
dictGetOrDefault test_01037.dict_tuple (0,2) Click North 422
dictGetOrDefault test_01037.dict_tuple (0.1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault test_01037.dict_tuple (1,0) Click East 421
dictGetOrDefault test_01037.dict_tuple (3,3) House 314159
dictGetOrDefault test_01037.dict_tuple (5,6) Click 42
dictGetOrDefault test_01037.dict_tuple (7.01,7.01) ee 55
dictHas test_01037.dict_array (-100,-42) 0
dictHas test_01037.dict_array (-1,0) 1
dictHas test_01037.dict_array (-0.1,0) 1
dictHas test_01037.dict_array (0,-2) 1
dictHas test_01037.dict_array (0,-1.1) 1
dictHas test_01037.dict_array (0,1.1) 1
dictHas test_01037.dict_array (0,2) 1
dictHas test_01037.dict_array (0.1,0) 1
dictHas test_01037.dict_array (0.99,2.99) 1
dictHas test_01037.dict_array (1,0) 1
dictHas test_01037.dict_array (3,3) 1
dictHas test_01037.dict_array (5,6) 1
dictHas test_01037.dict_array (7.01,7.01) 0
dictHas test_01037.dict_tuple (-100,-42) 0
dictHas test_01037.dict_tuple (-1,0) 1
dictHas test_01037.dict_tuple (-0.1,0) 1
dictHas test_01037.dict_tuple (0,-2) 1
dictHas test_01037.dict_tuple (0,-1.1) 1
dictHas test_01037.dict_tuple (0,1.1) 1
dictHas test_01037.dict_tuple (0,2) 1
dictHas test_01037.dict_tuple (0.1,0) 1
dictHas test_01037.dict_tuple (0.99,2.99) 1
dictHas test_01037.dict_tuple (1,0) 1
dictHas test_01037.dict_tuple (3,3) 1
dictHas test_01037.dict_tuple (5,6) 1
dictHas test_01037.dict_tuple (7.01,7.01) 0
dictGet dict_array (-100,-42) qqq 101
dictGet dict_array (-1,0) Click South 423
dictGet dict_array (-0.1,0) Click South 423
dictGet dict_array (0,-2) Click West 424
dictGet dict_array (0,-1.1) Click West 424
dictGet dict_array (0,1.1) Click North 422
dictGet dict_array (0,2) Click North 422
dictGet dict_array (0.1,0) Click East 421
dictGet dict_array (0.99,2.99) Click North 422
dictGet dict_array (1,0) Click East 421
dictGet dict_array (3,3) House 314159
dictGet dict_array (5,6) Click 42
dictGet dict_array (7.01,7.01) qqq 101
dictGetOrDefault dict_array (-100,-42) www 1234
dictGetOrDefault dict_array (-1,0) Click South 423
dictGetOrDefault dict_array (-0.1,0) Click South 423
dictGetOrDefault dict_array (0,-2) Click West 424
dictGetOrDefault dict_array (0,-1.1) Click West 424
dictGetOrDefault dict_array (0,1.1) Click North 422
dictGetOrDefault dict_array (0,2) Click North 422
dictGetOrDefault dict_array (0.1,0) Click East 421
dictGetOrDefault dict_array (0.99,2.99) Click North 422
dictGetOrDefault dict_array (1,0) Click East 421
dictGetOrDefault dict_array (3,3) House 314159
dictGetOrDefault dict_array (5,6) Click 42
dictGetOrDefault dict_array (7.01,7.01) www 1234
dictGetOrDefault dict_array (-100,-42) dd 44
dictGetOrDefault dict_array (-1,0) Click South 423
dictGetOrDefault dict_array (-0.1,0) Click South 423
dictGetOrDefault dict_array (0,-2) Click West 424
dictGetOrDefault dict_array (0,-1.1) Click West 424
dictGetOrDefault dict_array (0,1.1) Click North 422
dictGetOrDefault dict_array (0,2) Click North 422
dictGetOrDefault dict_array (0.1,0) Click East 421
dictGetOrDefault dict_array (0.99,2.99) Click North 422
dictGetOrDefault dict_array (1,0) Click East 421
dictGetOrDefault dict_array (3,3) House 314159
dictGetOrDefault dict_array (5,6) Click 42
dictGetOrDefault dict_array (7.01,7.01) ee 55
dictGet dict_tuple (-100,-42) qqq 101
dictGet dict_tuple (-1,0) Click South 423
dictGet dict_tuple (-0.1,0) Click South 423
dictGet dict_tuple (0,-2) Click West 424
dictGet dict_tuple (0,-1.1) Click West 424
dictGet dict_tuple (0,1.1) Click North 422
dictGet dict_tuple (0,2) Click North 422
dictGet dict_tuple (0.1,0) Click East 421
dictGet dict_tuple (0.99,2.99) Click North 422
dictGet dict_tuple (1,0) Click East 421
dictGet dict_tuple (3,3) House 314159
dictGet dict_tuple (5,6) Click 42
dictGet dict_tuple (7.01,7.01) qqq 101
dictGetOrDefault dict_tuple (-100,-42) www 1234
dictGetOrDefault dict_tuple (-1,0) Click South 423
dictGetOrDefault dict_tuple (-0.1,0) Click South 423
dictGetOrDefault dict_tuple (0,-2) Click West 424
dictGetOrDefault dict_tuple (0,-1.1) Click West 424
dictGetOrDefault dict_tuple (0,1.1) Click North 422
dictGetOrDefault dict_tuple (0,2) Click North 422
dictGetOrDefault dict_tuple (0.1,0) Click East 421
dictGetOrDefault dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault dict_tuple (1,0) Click East 421
dictGetOrDefault dict_tuple (3,3) House 314159
dictGetOrDefault dict_tuple (5,6) Click 42
dictGetOrDefault dict_tuple (7.01,7.01) www 1234
dictGetOrDefault dict_tuple (-100,-42) dd 44
dictGetOrDefault dict_tuple (-1,0) Click South 423
dictGetOrDefault dict_tuple (-0.1,0) Click South 423
dictGetOrDefault dict_tuple (0,-2) Click West 424
dictGetOrDefault dict_tuple (0,-1.1) Click West 424
dictGetOrDefault dict_tuple (0,1.1) Click North 422
dictGetOrDefault dict_tuple (0,2) Click North 422
dictGetOrDefault dict_tuple (0.1,0) Click East 421
dictGetOrDefault dict_tuple (0.99,2.99) Click North 422
dictGetOrDefault dict_tuple (1,0) Click East 421
dictGetOrDefault dict_tuple (3,3) House 314159
dictGetOrDefault dict_tuple (5,6) Click 42
dictGetOrDefault dict_tuple (7.01,7.01) ee 55
dictHas dict_array (-100,-42) 0
dictHas dict_array (-1,0) 1
dictHas dict_array (-0.1,0) 1
dictHas dict_array (0,-2) 1
dictHas dict_array (0,-1.1) 1
dictHas dict_array (0,1.1) 1
dictHas dict_array (0,2) 1
dictHas dict_array (0.1,0) 1
dictHas dict_array (0.99,2.99) 1
dictHas dict_array (1,0) 1
dictHas dict_array (3,3) 1
dictHas dict_array (5,6) 1
dictHas dict_array (7.01,7.01) 0
dictHas dict_tuple (-100,-42) 0
dictHas dict_tuple (-1,0) 1
dictHas dict_tuple (-0.1,0) 1
dictHas dict_tuple (0,-2) 1
dictHas dict_tuple (0,-1.1) 1
dictHas dict_tuple (0,1.1) 1
dictHas dict_tuple (0,2) 1
dictHas dict_tuple (0.1,0) 1
dictHas dict_tuple (0.99,2.99) 1
dictHas dict_tuple (1,0) 1
dictHas dict_tuple (3,3) 1
dictHas dict_tuple (5,6) 1
dictHas dict_tuple (7.01,7.01) 0

View File

@ -1,56 +1,52 @@
#!/usr/bin/env bash
# Tags: no-debug, no-parallel
# Tags: no-debug
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
TMP_DIR="/tmp"
TMP_DIR=${CLICKHOUSE_TMP}/tmp
mkdir -p $TMP_DIR
$CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS test_01037;
DROP TABLE IF EXISTS polygons_array;
CREATE DATABASE test_01037;
CREATE TABLE polygons_array (key Array(Array(Array(Array(Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO polygons_array VALUES ([[[[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]], [[[5, 5], [5, 1], [7, 1], [7, 7], [1, 7], [1, 5]]]], 'Click', 42);
INSERT INTO polygons_array VALUES ([[[[5, 5], [5, -5], [-5, -5], [-5, 5]], [[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]]], 'House', 314159);
INSERT INTO polygons_array VALUES ([[[[3, 1], [0, 1], [0, -1], [3, -1]]]], 'Click East', 421);
INSERT INTO polygons_array VALUES ([[[[-1, 1], [1, 1], [1, 3], [-1, 3]]]], 'Click North', 422);
INSERT INTO polygons_array VALUES ([[[[-3, 1], [-3, -1], [0, -1], [0, 1]]]], 'Click South', 423);
INSERT INTO polygons_array VALUES ([[[[-1, -1], [1, -1], [1, -3], [-1, -3]]]], 'Click West', 424);
DROP TABLE IF EXISTS test_01037.polygons_array;
DROP TABLE IF EXISTS polygons_tuple;
CREATE TABLE test_01037.polygons_array (key Array(Array(Array(Array(Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO test_01037.polygons_array VALUES ([[[[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]], [[[5, 5], [5, 1], [7, 1], [7, 7], [1, 7], [1, 5]]]], 'Click', 42);
INSERT INTO test_01037.polygons_array VALUES ([[[[5, 5], [5, -5], [-5, -5], [-5, 5]], [[1, 3], [1, 1], [3, 1], [3, -1], [1, -1], [1, -3], [-1, -3], [-1, -1], [-3, -1], [-3, 1], [-1, 1], [-1, 3]]]], 'House', 314159);
INSERT INTO test_01037.polygons_array VALUES ([[[[3, 1], [0, 1], [0, -1], [3, -1]]]], 'Click East', 421);
INSERT INTO test_01037.polygons_array VALUES ([[[[-1, 1], [1, 1], [1, 3], [-1, 3]]]], 'Click North', 422);
INSERT INTO test_01037.polygons_array VALUES ([[[[-3, 1], [-3, -1], [0, -1], [0, 1]]]], 'Click South', 423);
INSERT INTO test_01037.polygons_array VALUES ([[[[-1, -1], [1, -1], [1, -3], [-1, -3]]]], 'Click West', 424);
CREATE TABLE polygons_tuple (key Array(Array(Array(Tuple(Float64, Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO polygons_tuple VALUES ([[[(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]], [[(5, 5), (5, 1), (7, 1), (7, 7), (1, 7), (1, 5)]]], 'Click', 42);
INSERT INTO polygons_tuple VALUES ([[[(5, 5), (5, -5), (-5, -5), (-5, 5)], [(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]]], 'House', 314159);
INSERT INTO polygons_tuple VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Click East', 421);
INSERT INTO polygons_tuple VALUES ([[[(-1, 1), (1, 1), (1, 3), (-1, 3)]]], 'Click North', 422);
INSERT INTO polygons_tuple VALUES ([[[(-3, 1), (-3, -1), (0, -1), (0, 1)]]], 'Click South', 423);
INSERT INTO polygons_tuple VALUES ([[[(-1, -1), (1, -1), (1, -3), (-1, -3)]]], 'Click West', 424);
DROP TABLE IF EXISTS test_01037.polygons_tuple;
DROP TABLE IF EXISTS points;
CREATE TABLE test_01037.polygons_tuple (key Array(Array(Array(Tuple(Float64, Float64)))), name String, value UInt64) ENGINE = Memory;
INSERT INTO test_01037.polygons_tuple VALUES ([[[(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]], [[(5, 5), (5, 1), (7, 1), (7, 7), (1, 7), (1, 5)]]], 'Click', 42);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(5, 5), (5, -5), (-5, -5), (-5, 5)], [(1, 3), (1, 1), (3, 1), (3, -1), (1, -1), (1, -3), (-1, -3), (-1, -1), (-3, -1), (-3, 1), (-1, 1), (-1, 3)]]], 'House', 314159);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Click East', 421);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-1, 1), (1, 1), (1, 3), (-1, 3)]]], 'Click North', 422);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-3, 1), (-3, -1), (0, -1), (0, 1)]]], 'Click South', 423);
INSERT INTO test_01037.polygons_tuple VALUES ([[[(-1, -1), (1, -1), (1, -3), (-1, -3)]]], 'Click West', 424);
DROP TABLE IF EXISTS test_01037.points;
CREATE TABLE test_01037.points (x Float64, y Float64, def_i UInt64, def_s String) ENGINE = Memory;
INSERT INTO test_01037.points VALUES (0.1, 0.0, 112, 'aax');
INSERT INTO test_01037.points VALUES (-0.1, 0.0, 113, 'aay');
INSERT INTO test_01037.points VALUES (0.0, 1.1, 114, 'aaz');
INSERT INTO test_01037.points VALUES (0.0, -1.1, 115, 'aat');
INSERT INTO test_01037.points VALUES (3.0, 3.0, 22, 'bb');
INSERT INTO test_01037.points VALUES (5.0, 6.0, 33, 'cc');
INSERT INTO test_01037.points VALUES (-100.0, -42.0, 44, 'dd');
INSERT INTO test_01037.points VALUES (7.01, 7.01, 55, 'ee');
INSERT INTO test_01037.points VALUES (0.99, 2.99, 66, 'ee');
INSERT INTO test_01037.points VALUES (1.0, 0.0, 771, 'ffa');
INSERT INTO test_01037.points VALUES (-1.0, 0.0, 772, 'ffb');
INSERT INTO test_01037.points VALUES (0.0, 2.0, 773, 'ffc');
INSERT INTO test_01037.points VALUES (0.0, -2.0, 774, 'ffd');
CREATE TABLE points (x Float64, y Float64, def_i UInt64, def_s String) ENGINE = Memory;
INSERT INTO points VALUES (0.1, 0.0, 112, 'aax');
INSERT INTO points VALUES (-0.1, 0.0, 113, 'aay');
INSERT INTO points VALUES (0.0, 1.1, 114, 'aaz');
INSERT INTO points VALUES (0.0, -1.1, 115, 'aat');
INSERT INTO points VALUES (3.0, 3.0, 22, 'bb');
INSERT INTO points VALUES (5.0, 6.0, 33, 'cc');
INSERT INTO points VALUES (-100.0, -42.0, 44, 'dd');
INSERT INTO points VALUES (7.01, 7.01, 55, 'ee');
INSERT INTO points VALUES (0.99, 2.99, 66, 'ee');
INSERT INTO points VALUES (1.0, 0.0, 771, 'ffa');
INSERT INTO points VALUES (-1.0, 0.0, 772, 'ffb');
INSERT INTO points VALUES (0.0, 2.0, 773, 'ffc');
INSERT INTO points VALUES (0.0, -2.0, 774, 'ffd');
"
declare -a SearchTypes=("POLYGON" "POLYGON_SIMPLE" "POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL")
for type in "${SearchTypes[@]}";
@ -58,63 +54,62 @@ do
outputFile="${TMP_DIR}/results${type}.out"
$CLICKHOUSE_CLIENT -n --query="
DROP DICTIONARY IF EXISTS test_01037.dict_array;
CREATE DICTIONARY test_01037.dict_array
DROP DICTIONARY IF EXISTS dict_array;
CREATE DICTIONARY dict_array
(
key Array(Array(Array(Array(Float64)))),
name String DEFAULT 'qqq',
value UInt64 DEFAULT 101
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_array' PASSWORD '' DB 'test_01037'))
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_array' PASSWORD '' DB currentDatabase()))
LIFETIME(0)
LAYOUT($type());
DROP DICTIONARY IF EXISTS test_01037.dict_tuple;
DROP DICTIONARY IF EXISTS dict_tuple;
CREATE DICTIONARY test_01037.dict_tuple
CREATE DICTIONARY dict_tuple
(
key Array(Array(Array(Tuple(Float64, Float64)))),
name String DEFAULT 'qqq',
value UInt64 DEFAULT 101
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_tuple' PASSWORD '' DB 'test_01037'))
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_tuple' PASSWORD '' DB currentDatabase()))
LIFETIME(0)
LAYOUT($type());
select 'dictGet', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
select 'dictGet', 'dict_array' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'name', key),
dictGet(dict_name, 'value', key) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'value', key) from points order by x, y;
select 'dictGetOrDefault', 'dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, 'www'),
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from points order by x, y;
select 'dictGetOrDefault', 'dict_array' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, def_s),
dictGetOrDefault(dict_name, 'value', key, def_i) from test_01037.points order by x, y;
select 'dictGet', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'value', key, def_i) from points order by x, y;
select 'dictGet', 'dict_tuple' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'name', key),
dictGet(dict_name, 'value', key) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGet(dict_name, 'value', key) from points order by x, y;
select 'dictGetOrDefault', 'dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, 'www'),
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from test_01037.points order by x, y;
select 'dictGetOrDefault', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'value', key, toUInt64(1234)) from points order by x, y;
select 'dictGetOrDefault', 'dict_tuple' as dict_name, tuple(x, y) as key,
dictGetOrDefault(dict_name, 'name', key, def_s),
dictGetOrDefault(dict_name, 'value', key, def_i) from test_01037.points order by x, y;
select 'dictHas', 'test_01037.dict_array' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from test_01037.points order by x, y;
select 'dictHas', 'test_01037.dict_tuple' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from test_01037.points order by x, y;
dictGetOrDefault(dict_name, 'value', key, def_i) from points order by x, y;
select 'dictHas', 'dict_array' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from points order by x, y;
select 'dictHas', 'dict_tuple' as dict_name, tuple(x, y) as key,
dictHas(dict_name, key) from points order by x, y;
" > "$outputFile"
diff -q "${CURDIR}/01037_polygon_dicts_simple_functions.ans" "$outputFile"
done
$CLICKHOUSE_CLIENT -n --query="
DROP DICTIONARY test_01037.dict_array;
DROP DICTIONARY test_01037.dict_tuple;
DROP TABLE test_01037.polygons_array;
DROP TABLE test_01037.polygons_tuple;
DROP TABLE test_01037.points;
DROP DATABASE test_01037;
DROP DICTIONARY dict_array;
DROP DICTIONARY dict_tuple;
DROP TABLE polygons_array;
DROP TABLE polygons_tuple;
DROP TABLE points;
"

View File

@ -1,4 +1,4 @@
-- Tags: no-fasttest
-- This tests depends on internet access, but it does not matter, because it only has to check that there is no abort due to a bug in Apache Arrow library.
SET optimize_trivial_insert_select=1;
INSERT INTO TABLE FUNCTION url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet') SELECT * FROM url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet'); -- { serverError CANNOT_WRITE_TO_OSTREAM, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, POCO_EXCEPTION }