Merge remote-tracking branch 'origin/json-utc-formatted-log' into json-utc-formatted-log

This commit is contained in:
xogoodnow 2024-11-07 14:01:00 +03:30
commit c7e2ae208d
27 changed files with 273 additions and 270 deletions

View File

@ -24,92 +24,7 @@ namespace ErrorCodes
void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
{
if (!ast)
{
chassert(false);
return;
}
/// FIXME: this helper should use updatePointerToChild(), but
/// forEachPointerToChild() is not implemented for ASTColumnDeclaration
/// (and also some members should be adjusted for this).
const auto visit_child_with_shared_ptr = [&](ASTPtr & child)
{
if (!child)
return;
auto * old_value = child.get();
visit(child);
// child did not change
if (old_value == child.get())
return;
// child changed, we need to modify it in the list of children of the parent also
for (auto & current_child : ast->children)
{
if (current_child.get() == old_value)
current_child = child;
}
};
if (auto * col_decl = ast->as<ASTColumnDeclaration>())
{
visit_child_with_shared_ptr(col_decl->default_expression);
visit_child_with_shared_ptr(col_decl->ttl);
return;
}
if (auto * storage = ast->as<ASTStorage>())
{
const auto visit_child = [&](IAST * & child)
{
if (!child)
return;
if (const auto * function = child->template as<ASTFunction>())
{
std::unordered_set<std::string> udf_in_replace_process;
auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process);
if (replace_result)
ast->setOrReplace(child, replace_result);
}
visit(child);
};
visit_child(storage->partition_by);
visit_child(storage->primary_key);
visit_child(storage->order_by);
visit_child(storage->sample_by);
visit_child(storage->ttl_table);
return;
}
if (auto * alter = ast->as<ASTAlterCommand>())
{
/// It is OK to use updatePointerToChild() because ASTAlterCommand implements forEachPointerToChild()
const auto visit_child_update_parent = [&](ASTPtr & child)
{
if (!child)
return;
auto * old_ptr = child.get();
visit(child);
auto * new_ptr = child.get();
/// Some AST classes have naked pointers to children elements as members.
/// We have to replace them if the child was replaced.
if (new_ptr != old_ptr)
ast->updatePointerToChild(old_ptr, new_ptr);
};
for (auto & children : alter->children)
visit_child_update_parent(children);
return;
}
chassert(ast);
if (const auto * function = ast->template as<ASTFunction>())
{
@ -120,7 +35,19 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
}
for (auto & child : ast->children)
{
if (!child)
return;
auto * old_ptr = child.get();
visit(child);
auto * new_ptr = child.get();
/// Some AST classes have naked pointers to children elements as members.
/// We have to replace them if the child was replaced.
if (new_ptr != old_ptr)
ast->updatePointerToChild(old_ptr, new_ptr);
}
}
void UserDefinedSQLFunctionVisitor::visit(IAST * ast)

View File

@ -128,4 +128,14 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo
}
}
void ASTColumnDeclaration::forEachPointerToChild(std::function<void(void **)> f)
{
f(reinterpret_cast<void **>(&default_expression));
f(reinterpret_cast<void **>(&comment));
f(reinterpret_cast<void **>(&codec));
f(reinterpret_cast<void **>(&statistics_desc));
f(reinterpret_cast<void **>(&ttl));
f(reinterpret_cast<void **>(&collation));
f(reinterpret_cast<void **>(&settings));
}
}

View File

@ -29,6 +29,9 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const override;
protected:
void forEachPointerToChild(std::function<void(void **)> f) override;
};
}

View File

@ -345,10 +345,11 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_
throw Exception(ErrorCodes::INCORRECT_DATA, "Index granularity is too big: more than {} rows per index granule.", std::numeric_limits<UInt32>::max());
if (index_sample_block.columns() > 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected that index is build over a single column");
const String & index_column_name = index_sample_block.getByPosition(0).name;
const ColumnPtr & index_column = block.getByName(index_column_name).column;
const auto & index_column_name = index_sample_block.getByPosition(0).name;
const auto & index_column = block.getByName(index_column_name).column;
ColumnPtr column_cut = index_column->cut(*pos, rows_read);
const auto * column_array = typeid_cast<const ColumnArray *>(column_cut.get());
@ -382,8 +383,7 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_
if (index->size() + rows > std::numeric_limits<UInt32>::max())
throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries");
DataTypePtr data_type = block.getDataTypes()[0];
const auto * data_type_array = typeid_cast<const DataTypeArray *>(data_type.get());
const auto * data_type_array = typeid_cast<const DataTypeArray *>(block.getByName(index_column_name).type.get());
if (!data_type_array)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)");
const TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId();

View File

@ -51,11 +51,11 @@ class CI:
TAG_CONFIGS = {
Tags.DO_NOT_TEST_LABEL: LabelConfig(run_jobs=[JobNames.STYLE_CHECK]),
Tags.CI_SET_ARM: LabelConfig(
Tags.CI_SET_AARCH64: LabelConfig(
run_jobs=[
JobNames.STYLE_CHECK,
BuildNames.PACKAGE_AARCH64,
JobNames.INTEGRATION_TEST_ARM,
JobNames.INTEGRATION_TEST_AARCH64,
]
),
Tags.CI_SET_REQUIRED: LabelConfig(
@ -95,16 +95,16 @@ class CI:
static_binary_name="aarch64",
additional_pkgs=True,
),
runner_type=Runners.BUILDER_ARM,
runner_type=Runners.BUILDER_AARCH64,
),
BuildNames.PACKAGE_ARM_ASAN: CommonJobConfigs.BUILD.with_properties(
BuildNames.PACKAGE_AARCH64_ASAN: CommonJobConfigs.BUILD.with_properties(
build_config=BuildConfig(
name=BuildNames.PACKAGE_ARM_ASAN,
name=BuildNames.PACKAGE_AARCH64_ASAN,
compiler="clang-18-aarch64",
sanitizer="address",
package_type="deb",
),
runner_type=Runners.BUILDER_ARM,
runner_type=Runners.BUILDER_AARCH64,
),
BuildNames.PACKAGE_ASAN: CommonJobConfigs.BUILD.with_properties(
build_config=BuildConfig(
@ -276,16 +276,16 @@ class CI:
JobNames.INSTALL_TEST_AMD: CommonJobConfigs.INSTALL_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE]
),
JobNames.INSTALL_TEST_ARM: CommonJobConfigs.INSTALL_TEST.with_properties(
JobNames.INSTALL_TEST_AARCH64: CommonJobConfigs.INSTALL_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64],
runner_type=Runners.STYLE_CHECKER_ARM,
runner_type=Runners.STYLE_CHECKER_AARCH64,
),
JobNames.STATEFUL_TEST_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN]
),
JobNames.STATEFUL_TEST_ARM_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ARM_ASAN],
runner_type=Runners.FUNC_TESTER_ARM,
JobNames.STATEFUL_TEST_AARCH64_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64_ASAN],
runner_type=Runners.FUNC_TESTER_AARCH64,
),
JobNames.STATEFUL_TEST_TSAN: CommonJobConfigs.STATEFUL_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_TSAN]
@ -307,7 +307,7 @@ class CI:
),
JobNames.STATEFUL_TEST_AARCH64: CommonJobConfigs.STATEFUL_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64],
runner_type=Runners.FUNC_TESTER_ARM,
runner_type=Runners.FUNC_TESTER_AARCH64,
),
JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: CommonJobConfigs.STATEFUL_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE]
@ -335,10 +335,10 @@ class CI:
JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2
),
JobNames.STATELESS_TEST_ARM_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ARM_ASAN],
JobNames.STATELESS_TEST_AARCH64_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64_ASAN],
num_batches=2,
runner_type=Runners.FUNC_TESTER_ARM,
runner_type=Runners.FUNC_TESTER_AARCH64,
),
JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_TSAN], num_batches=4
@ -360,7 +360,7 @@ class CI:
),
JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64],
runner_type=Runners.FUNC_TESTER_ARM,
runner_type=Runners.FUNC_TESTER_AARCH64,
),
JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=2
@ -432,10 +432,10 @@ class CI:
num_batches=6,
timeout=9000, # the job timed out with default value (7200)
),
JobNames.INTEGRATION_TEST_ARM: CommonJobConfigs.INTEGRATION_TEST.with_properties(
JobNames.INTEGRATION_TEST_AARCH64: CommonJobConfigs.INTEGRATION_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64],
num_batches=6,
runner_type=Runners.FUNC_TESTER_ARM,
runner_type=Runners.FUNC_TESTER_AARCH64,
),
JobNames.INTEGRATION_TEST: CommonJobConfigs.INTEGRATION_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE],
@ -453,10 +453,10 @@ class CI:
required_builds=[BuildNames.PACKAGE_RELEASE],
required_on_release_branch=True,
),
JobNames.COMPATIBILITY_TEST_ARM: CommonJobConfigs.COMPATIBILITY_TEST.with_properties(
JobNames.COMPATIBILITY_TEST_AARCH64: CommonJobConfigs.COMPATIBILITY_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64],
required_on_release_branch=True,
runner_type=Runners.STYLE_CHECKER_ARM,
runner_type=Runners.STYLE_CHECKER_AARCH64,
),
JobNames.UNIT_TEST: CommonJobConfigs.UNIT_TEST.with_properties(
required_builds=[BuildNames.BINARY_RELEASE],
@ -499,22 +499,22 @@ class CI:
required_builds=[BuildNames.BINARY_RELEASE],
run_by_labels=[Labels.JEPSEN_TEST],
run_command="jepsen_check.py keeper",
runner_type=Runners.STYLE_CHECKER_ARM,
runner_type=Runners.STYLE_CHECKER_AARCH64,
),
JobNames.JEPSEN_SERVER: JobConfig(
required_builds=[BuildNames.BINARY_RELEASE],
run_by_labels=[Labels.JEPSEN_TEST],
run_command="jepsen_check.py server",
runner_type=Runners.STYLE_CHECKER_ARM,
runner_type=Runners.STYLE_CHECKER_AARCH64,
),
JobNames.PERFORMANCE_TEST_AMD64: CommonJobConfigs.PERF_TESTS.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4
),
JobNames.PERFORMANCE_TEST_ARM64: CommonJobConfigs.PERF_TESTS.with_properties(
JobNames.PERFORMANCE_TEST_AARCH64: CommonJobConfigs.PERF_TESTS.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64],
num_batches=4,
run_by_labels=[Labels.PR_PERFORMANCE],
runner_type=Runners.FUNC_TESTER_ARM,
runner_type=Runners.FUNC_TESTER_AARCH64,
),
JobNames.SQLANCER: CommonJobConfigs.SQLLANCER_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE],
@ -532,9 +532,9 @@ class CI:
JobNames.CLICKBENCH_TEST: CommonJobConfigs.CLICKBENCH_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE],
),
JobNames.CLICKBENCH_TEST_ARM: CommonJobConfigs.CLICKBENCH_TEST.with_properties(
JobNames.CLICKBENCH_TEST_AARCH64: CommonJobConfigs.CLICKBENCH_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_AARCH64],
runner_type=Runners.FUNC_TESTER_ARM,
runner_type=Runners.FUNC_TESTER_AARCH64,
),
JobNames.LIBFUZZER_TEST: JobConfig(
required_builds=[BuildNames.FUZZERS],
@ -572,7 +572,7 @@ class CI:
),
JobNames.STYLE_CHECK: JobConfig(
run_always=True,
runner_type=Runners.STYLE_CHECKER_ARM,
runner_type=Runners.STYLE_CHECKER_AARCH64,
),
JobNames.BUGFIX_VALIDATE: JobConfig(
run_by_labels=[Labels.PR_BUGFIX, Labels.PR_CRITICAL_BUGFIX],

View File

@ -58,11 +58,11 @@ class Runners(metaclass=WithIter):
"""
BUILDER = "builder"
BUILDER_ARM = "builder-aarch64"
BUILDER_AARCH64 = "builder-aarch64"
STYLE_CHECKER = "style-checker"
STYLE_CHECKER_ARM = "style-checker-aarch64"
STYLE_CHECKER_AARCH64 = "style-checker-aarch64"
FUNC_TESTER = "func-tester"
FUNC_TESTER_ARM = "func-tester-aarch64"
FUNC_TESTER_AARCH64 = "func-tester-aarch64"
FUZZER_UNIT_TESTER = "fuzzer-unit-tester"
@ -78,7 +78,7 @@ class Tags(metaclass=WithIter):
# to upload all binaries from build jobs
UPLOAD_ALL_ARTIFACTS = "upload_all"
CI_SET_SYNC = "ci_set_sync"
CI_SET_ARM = "ci_set_arm"
CI_SET_AARCH64 = "ci_set_aarch64"
CI_SET_REQUIRED = "ci_set_required"
CI_SET_BUILDS = "ci_set_builds"
@ -106,7 +106,7 @@ class BuildNames(metaclass=WithIter):
PACKAGE_MSAN = "package_msan"
PACKAGE_DEBUG = "package_debug"
PACKAGE_AARCH64 = "package_aarch64"
PACKAGE_ARM_ASAN = "package_aarch64_asan"
PACKAGE_AARCH64_ASAN = "package_aarch64_asan"
PACKAGE_RELEASE_COVERAGE = "package_release_coverage"
BINARY_RELEASE = "binary_release"
BINARY_TIDY = "binary_tidy"
@ -134,14 +134,14 @@ class JobNames(metaclass=WithIter):
DOCKER_SERVER = "Docker server image"
DOCKER_KEEPER = "Docker keeper image"
INSTALL_TEST_AMD = "Install packages (release)"
INSTALL_TEST_ARM = "Install packages (aarch64)"
INSTALL_TEST_AARCH64 = "Install packages (aarch64)"
STATELESS_TEST_DEBUG = "Stateless tests (debug)"
STATELESS_TEST_RELEASE = "Stateless tests (release)"
STATELESS_TEST_RELEASE_COVERAGE = "Stateless tests (coverage)"
STATELESS_TEST_AARCH64 = "Stateless tests (aarch64)"
STATELESS_TEST_ASAN = "Stateless tests (asan)"
STATELESS_TEST_ARM_ASAN = "Stateless tests (aarch64, asan)"
STATELESS_TEST_AARCH64_ASAN = "Stateless tests (aarch64, asan)"
STATELESS_TEST_TSAN = "Stateless tests (tsan)"
STATELESS_TEST_MSAN = "Stateless tests (msan)"
STATELESS_TEST_UBSAN = "Stateless tests (ubsan)"
@ -158,7 +158,7 @@ class JobNames(metaclass=WithIter):
STATEFUL_TEST_RELEASE_COVERAGE = "Stateful tests (coverage)"
STATEFUL_TEST_AARCH64 = "Stateful tests (aarch64)"
STATEFUL_TEST_ASAN = "Stateful tests (asan)"
STATEFUL_TEST_ARM_ASAN = "Stateful tests (aarch64, asan)"
STATEFUL_TEST_AARCH64_ASAN = "Stateful tests (aarch64, asan)"
STATEFUL_TEST_TSAN = "Stateful tests (tsan)"
STATEFUL_TEST_MSAN = "Stateful tests (msan)"
STATEFUL_TEST_UBSAN = "Stateful tests (ubsan)"
@ -181,7 +181,7 @@ class JobNames(metaclass=WithIter):
INTEGRATION_TEST_ASAN = "Integration tests (asan)"
INTEGRATION_TEST_ASAN_OLD_ANALYZER = "Integration tests (asan, old analyzer)"
INTEGRATION_TEST_TSAN = "Integration tests (tsan)"
INTEGRATION_TEST_ARM = "Integration tests (aarch64)"
INTEGRATION_TEST_AARCH64 = "Integration tests (aarch64)"
INTEGRATION_TEST_FLAKY = "Integration tests flaky check (asan)"
UPGRADE_TEST_DEBUG = "Upgrade check (debug)"
@ -205,7 +205,7 @@ class JobNames(metaclass=WithIter):
JEPSEN_SERVER = "ClickHouse Server Jepsen"
PERFORMANCE_TEST_AMD64 = "Performance Comparison (release)"
PERFORMANCE_TEST_ARM64 = "Performance Comparison (aarch64)"
PERFORMANCE_TEST_AARCH64 = "Performance Comparison (aarch64)"
# SQL_LOGIC_TEST = "Sqllogic test (release)"
@ -214,10 +214,10 @@ class JobNames(metaclass=WithIter):
SQLTEST = "SQLTest"
COMPATIBILITY_TEST = "Compatibility check (release)"
COMPATIBILITY_TEST_ARM = "Compatibility check (aarch64)"
COMPATIBILITY_TEST_AARCH64 = "Compatibility check (aarch64)"
CLICKBENCH_TEST = "ClickBench (release)"
CLICKBENCH_TEST_ARM = "ClickBench (aarch64)"
CLICKBENCH_TEST_AARCH64 = "ClickBench (aarch64)"
LIBFUZZER_TEST = "libFuzzer tests"
@ -387,7 +387,7 @@ class CommonJobConfigs:
"./tests/ci/upload_result_helper.py",
],
),
runner_type=Runners.STYLE_CHECKER_ARM,
runner_type=Runners.STYLE_CHECKER_AARCH64,
disable_await=True,
)
COMPATIBILITY_TEST = JobConfig(
@ -634,8 +634,8 @@ REQUIRED_CHECKS = [
JobNames.STATEFUL_TEST_RELEASE,
JobNames.STATELESS_TEST_RELEASE,
JobNames.STATELESS_TEST_ASAN,
JobNames.STATELESS_TEST_ARM_ASAN,
JobNames.STATEFUL_TEST_ARM_ASAN,
JobNames.STATELESS_TEST_AARCH64_ASAN,
JobNames.STATEFUL_TEST_AARCH64_ASAN,
JobNames.STATELESS_TEST_FLAKY_ASAN,
JobNames.STATEFUL_TEST_ASAN,
JobNames.STYLE_CHECK,

View File

@ -131,7 +131,7 @@ def main():
check_name = args.check_name or os.getenv("CHECK_NAME")
assert check_name
check_glibc = True
# currently hardcoded to x86, don't enable for ARM
# currently hardcoded to x86, don't enable for AARCH64
check_distributions = (
"aarch64" not in check_name.lower() and "arm64" not in check_name.lower()
)

View File

@ -36,11 +36,12 @@ class TestCIConfig(unittest.TestCase):
elif "binary_" in job.lower() or "package_" in job.lower():
if job.lower() in (
CI.BuildNames.PACKAGE_AARCH64,
CI.BuildNames.PACKAGE_ARM_ASAN,
CI.BuildNames.PACKAGE_AARCH64_ASAN,
):
self.assertTrue(
CI.JOB_CONFIGS[job].runner_type in (CI.Runners.BUILDER_ARM,),
f"Job [{job}] must have [{CI.Runners.BUILDER_ARM}] runner",
CI.JOB_CONFIGS[job].runner_type
in (CI.Runners.BUILDER_AARCH64,),
f"Job [{job}] must have [{CI.Runners.BUILDER_AARCH64}] runner",
)
else:
self.assertTrue(
@ -96,7 +97,7 @@ class TestCIConfig(unittest.TestCase):
else:
self.assertTrue(CI.JOB_CONFIGS[job].build_config is None)
if "asan" in job and "aarch" in job:
expected_builds = [CI.BuildNames.PACKAGE_ARM_ASAN]
expected_builds = [CI.BuildNames.PACKAGE_AARCH64_ASAN]
elif "asan" in job:
expected_builds = [CI.BuildNames.PACKAGE_ASAN]
elif "msan" in job:

View File

@ -10,7 +10,7 @@ from ci_settings import CiSettings
_TEST_BODY_1 = """
#### Run only:
- [ ] <!---ci_set_required--> Some Set
- [x] <!---ci_set_arm--> Integration tests (arm64)
- [x] <!---ci_set_aarch64--> Integration tests (aarch64)
- [x] <!---ci_include_foo--> Integration tests
- [x] <!---ci_include_foo_Bar--> Integration tests
- [ ] <!---ci_include_bar--> Integration tests
@ -150,7 +150,7 @@ class TestCIOptions(unittest.TestCase):
self.assertFalse(ci_options.no_ci_cache)
self.assertTrue(ci_options.no_merge_commit)
self.assertTrue(ci_options.woolen_wolfdog)
self.assertEqual(ci_options.ci_sets, ["ci_set_arm"])
self.assertEqual(ci_options.ci_sets, ["ci_set_aarch64"])
self.assertCountEqual(ci_options.include_keywords, ["foo", "foo_bar"])
self.assertCountEqual(ci_options.exclude_keywords, ["foo", "foo_bar"])

View File

@ -1,8 +1,10 @@
import json
from datetime import datetime
from xml.etree import ElementTree as ET
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
@ -58,12 +60,21 @@ def validate_log_level(config, logs):
return True
def is_valid_utc_datetime(datetime_str):
try:
datetime_obj = datetime.strptime(datetime_str, "%Y-%m-%dT%H:%M:%S.%fZ")
return datetime_obj.tzinfo is None
except ValueError:
return False
def validate_log_config_relation(config, logs, config_type):
root = ET.fromstring(config)
keys_in_config = set()
if config_type == "config_no_keys":
keys_in_config.add("date_time")
keys_in_config.add("date_time_utc")
keys_in_config.add("thread_name")
keys_in_config.add("thread_id")
keys_in_config.add("level")
@ -85,9 +96,12 @@ def validate_log_config_relation(config, logs, config_type):
keys_in_log.add(log_key)
if log_key not in keys_in_config:
return False
for config_key in keys_in_config:
if config_key not in keys_in_log:
return False
# Validate the UTC datetime format in "date_time_utc" if it exists
if "date_time_utc" in json_log and not is_valid_utc_datetime(
json_log["date_time_utc"]
):
return False
except ValueError as e:
return False
return True

View File

@ -0,0 +1,13 @@
-- Tags: no-fasttest, no-ordinary-database
SET allow_experimental_vector_similarity_index = 1;
-- Issue #52258: Vector similarity indexes must reject empty Arrays or Arrays with default values
DROP TABLE IF EXISTS tab;
CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree() ORDER BY id;
INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA }
INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA }
DROP TABLE tab;

View File

@ -1,10 +1,3 @@
Rejects INSERTs of Arrays with different sizes
Issue #52258: Empty Arrays or Arrays with default values are rejected
It is possible to create parts with different Array vector sizes but there will be an error at query time
Correctness of index with > 1 mark
1 [1,0] 0
9000 [9000,0] 0
Issue #69085: Reference vector computed by a subquery
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
@ -40,4 +33,3 @@ Expression (Projection)
Condition: true
Parts: 1/1
Granules: 4/4
index_granularity_bytes = 0 is disallowed

View File

@ -0,0 +1,52 @@
-- Tags: no-fasttest, no-ordinary-database
SET allow_experimental_vector_similarity_index = 1;
SET enable_analyzer = 0;
-- Issue #69085: Reference vector for vector search is computed by a subquery
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
-- works
EXPLAIN indexes = 1
WITH [0., 2.] AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1;
-- does not work
EXPLAIN indexes = 1
WITH (
SELECT vec
FROM tab
LIMIT 1
) AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1;
-- does not work as well
EXPLAIN indexes = 1
WITH (
SELECT [0., 2.]
) AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1;
DROP TABLE tab;

View File

@ -0,0 +1,20 @@
-- Tags: no-fasttest, no-ordinary-database
SET allow_experimental_vector_similarity_index = 1;
-- Issue #71381: Usage of vector similarity index and further skipping indexes on the same table
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(
val String,
vec Array(Float32),
INDEX ann_idx vec TYPE vector_similarity('hnsw', 'cosineDistance'),
INDEX set_idx val TYPE set(100)
)
ENGINE = MergeTree()
ORDER BY tuple();
INSERT INTO tab VALUES ('hello world', [0.0]);
DROP TABLE tab;

View File

@ -0,0 +1,20 @@
-- Tags: no-fasttest, no-ordinary-database
-- Tests that vector similarity indexes cannot be created with index_granularity_bytes = 0
SET allow_experimental_vector_similarity_index = 1;
DROP TABLE IF EXISTS tab;
-- If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs.
-- SET allow_experimental_vector_similarity_index = 1;
-- CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0;
-- INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000);
-- WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100;
-- As a workaround, force enabled adaptive index granularity for now (it is the default anyways).
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; -- { serverError INVALID_SETTING_VALUE }
CREATE TABLE tab(id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0;
ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError INVALID_SETTING_VALUE }
DROP TABLE tab;

View File

@ -0,0 +1,24 @@
-- Tags: no-fasttest, no-ordinary-database
SET allow_experimental_vector_similarity_index = 1;
SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id;
-- Vector similarity indexes reject INSERTs of Arrays with different sizes
INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA }
-- It is possible to create parts with different Array vector sizes but there will be an error at query time
SYSTEM STOP MERGES tab;
INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]);
INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]);
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3; -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
DROP TABLE tab;

View File

@ -0,0 +1,2 @@
1 [1,0] 0
9000 [9000,0] 0

View File

@ -0,0 +1,25 @@
-- Tags: no-fasttest, no-ordinary-database
-- Tests correctness of vector similarity index with > 1 mark
SET allow_experimental_vector_similarity_index = 1;
SET enable_analyzer = 0;
DROP TABLE IF EXISTS tab;
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192;
INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000);
WITH [1.0, 0.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab
ORDER BY L2Distance(vec, reference_vec)
LIMIT 1;
WITH [9000.0, 0.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab
ORDER BY L2Distance(vec, reference_vec)
LIMIT 1;
DROP TABLE tab;

View File

@ -1,119 +0,0 @@
-- Tags: no-fasttest, no-ordinary-database
-- Tests various bugs and special cases for vector indexes.
SET allow_experimental_vector_similarity_index = 1;
SET enable_analyzer = 1; -- 0 vs. 1 produce slightly different error codes, make it future-proof
DROP TABLE IF EXISTS tab;
SELECT 'Rejects INSERTs of Arrays with different sizes';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id;
INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2, 3.3]); -- { serverError INCORRECT_DATA }
DROP TABLE tab;
SELECT 'Issue #52258: Empty Arrays or Arrays with default values are rejected';
CREATE TABLE tab (id UInt64, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree() ORDER BY id;
INSERT INTO tab VALUES (1, []); -- { serverError INCORRECT_DATA }
INSERT INTO tab (id) VALUES (1); -- { serverError INCORRECT_DATA }
DROP TABLE tab;
SELECT 'It is possible to create parts with different Array vector sizes but there will be an error at query time';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id;
SYSTEM STOP MERGES tab;
INSERT INTO tab values (0, [2.2, 2.3]) (1, [3.1, 3.2]);
INSERT INTO tab values (2, [2.2, 2.3, 2.4]) (3, [3.1, 3.2, 3.3]);
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3; -- { serverError SIZES_OF_ARRAYS_DONT_MATCH }
DROP TABLE tab;
SELECT 'Correctness of index with > 1 mark';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192;
INSERT INTO tab SELECT number, [toFloat32(number), 0.0] from numbers(10000);
WITH [1.0, 0.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab
ORDER BY L2Distance(vec, reference_vec)
LIMIT 1;
WITH [9000.0, 0.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab
ORDER BY L2Distance(vec, reference_vec)
LIMIT 1;
DROP TABLE tab;
SELECT 'Issue #69085: Reference vector computed by a subquery';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
-- works
EXPLAIN indexes = 1
WITH [0., 2.] AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1
SETTINGS enable_analyzer = 0;
-- does not work
EXPLAIN indexes = 1
WITH (
SELECT vec
FROM tab
LIMIT 1
) AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1
SETTINGS enable_analyzer = 0;
-- does not work as well
EXPLAIN indexes = 1
WITH (
SELECT [0., 2.]
) AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1
SETTINGS enable_analyzer = 0;
DROP TABLE tab;
SELECT 'index_granularity_bytes = 0 is disallowed';
-- If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs.
-- SET allow_experimental_vector_similarity_index = 1;
-- CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0;
-- INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000);
-- WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100;
-- As a workaround, force enabled adaptive index granularity for now (it is the default anyways).
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance')) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; -- { serverError INVALID_SETTING_VALUE }
CREATE TABLE tab(id Int32, vec Array(Float32)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0;
ALTER TABLE tab ADD INDEX vec_idx1(vec) TYPE vector_similarity('hnsw', 'cosineDistance'); -- { serverError INVALID_SETTING_VALUE }
DROP TABLE tab;

View File

@ -0,0 +1,2 @@
CREATE TABLE default.t0\n(\n `c0` Int32,\n CONSTRAINT c1 CHECK c0 > 5\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
10

View File

@ -0,0 +1,17 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "
CREATE FUNCTION ${CLICKHOUSE_DATABASE}_function AS (x) -> x > 5;
CREATE TABLE t0 (c0 Int, CONSTRAINT c1 CHECK ${CLICKHOUSE_DATABASE}_function(c0)) ENGINE = MergeTree() ORDER BY tuple();
SHOW CREATE TABLE t0;
INSERT INTO t0(c0) VALUES (10);
INSERT INTO t0(c0) VALUES (3); -- {serverError VIOLATED_CONSTRAINT}
SELECT * FROM t0;
DROP TABLE t0;
DROP FUNCTION ${CLICKHOUSE_DATABASE}_function;
"