Merge remote-tracking branch 'origin/master' into pr-local-plan

This commit is contained in:
Igor Nikonov 2024-09-04 11:15:54 +00:00
commit 3898f52868
17 changed files with 327 additions and 15 deletions

View File

@ -1287,7 +1287,8 @@ void Planner::buildPlanForUnionNode()
for (const auto & query_node : union_queries_nodes) for (const auto & query_node : union_queries_nodes)
{ {
Planner query_planner(query_node, select_query_options); Planner query_planner(query_node, select_query_options, planner_context->getGlobalPlannerContext());
query_planner.buildQueryPlanIfNeeded(); query_planner.buildQueryPlanIfNeeded();
for (const auto & row_policy : query_planner.getUsedRowPolicies()) for (const auto & row_policy : query_planner.getUsedRowPolicies())
used_row_policies.insert(row_policy); used_row_policies.insert(row_policy);

View File

@ -173,7 +173,10 @@ Pipe ReadFromMemoryStorageStep::makePipe()
for (size_t stream = 0; stream < num_streams; ++stream) for (size_t stream = 0; stream < num_streams; ++stream)
{ {
pipes.emplace_back(std::make_shared<MemorySource>(columns_to_read, storage_snapshot, current_data, parallel_execution_index)); auto source = std::make_shared<MemorySource>(columns_to_read, storage_snapshot, current_data, parallel_execution_index);
if (stream == 0)
source->addTotalRowsApprox(snapshot_data.rows_approx);
pipes.emplace_back(std::move(source));
} }
return Pipe::unitePipes(std::move(pipes)); return Pipe::unitePipes(std::move(pipes));
} }

View File

@ -161,6 +161,9 @@ StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr &
{ {
auto snapshot_data = std::make_unique<SnapshotData>(); auto snapshot_data = std::make_unique<SnapshotData>();
snapshot_data->blocks = data.get(); snapshot_data->blocks = data.get();
/// Not guaranteed to match `blocks`, but that's ok. It would probably be better to move
/// rows and bytes counters into the MultiVersion-ed struct, then everything would be consistent.
snapshot_data->rows_approx = total_size_rows.load(std::memory_order_relaxed);
if (!hasDynamicSubcolumns(metadata_snapshot->getColumns())) if (!hasDynamicSubcolumns(metadata_snapshot->getColumns()))
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, ColumnsDescription{}, std::move(snapshot_data)); return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, ColumnsDescription{}, std::move(snapshot_data));

View File

@ -42,6 +42,7 @@ public:
struct SnapshotData : public StorageSnapshot::Data struct SnapshotData : public StorageSnapshot::Data
{ {
std::shared_ptr<const Blocks> blocks; std::shared_ptr<const Blocks> blocks;
size_t rows_approx = 0;
}; };
StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override; StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override;

View File

@ -16,7 +16,7 @@ import upload_result_helper
from build_check import get_release_or_pr from build_check import get_release_or_pr
from ci_config import CI from ci_config import CI
from ci_metadata import CiMetadata from ci_metadata import CiMetadata
from ci_utils import GH, Utils from ci_utils import GH, Utils, Envs
from clickhouse_helper import ( from clickhouse_helper import (
CiLogsCredentials, CiLogsCredentials,
ClickHouseHelper, ClickHouseHelper,
@ -333,11 +333,10 @@ def _pre_action(s3, job_name, batch, indata, pr_info):
CI.JobNames.BUILD_CHECK, CI.JobNames.BUILD_CHECK,
): # we might want to rerun build report job ): # we might want to rerun build report job
rerun_helper = RerunHelper(commit, _get_ext_check_name(job_name)) rerun_helper = RerunHelper(commit, _get_ext_check_name(job_name))
if ( if rerun_helper.is_already_finished_by_status():
rerun_helper.is_already_finished_by_status() print(
and not Utils.is_job_triggered_manually() f"WARNING: Rerunning job with GH status, rerun triggered by {Envs.GITHUB_ACTOR}"
): )
print("WARNING: Rerunning job with GH status ")
status = rerun_helper.get_finished_status() status = rerun_helper.get_finished_status()
assert status assert status
print("::group::Commit Status") print("::group::Commit Status")

View File

@ -241,7 +241,10 @@ def main():
additional_data = [] additional_data = []
try: try:
test_result = _parse_jepsen_output(jepsen_log_path) test_result = _parse_jepsen_output(jepsen_log_path)
if any(r.status == "FAIL" for r in test_result): if len(test_result) == 0:
status = FAILURE
description = "No test results found"
elif any(r.status == "FAIL" for r in test_result):
status = FAILURE status = FAILURE
description = "Found invalid analysis (ノಥ益ಥ)ノ ┻━┻" description = "Found invalid analysis (ノಥ益ಥ)ノ ┻━┻"

View File

@ -1638,6 +1638,19 @@ class TestCase:
if args.client_log: if args.client_log:
log_opt = " --client_logs_file=" + args.client_log + " " log_opt = " --client_logs_file=" + args.client_log + " "
client_options += log_opt client_options += log_opt
for env_name in [
"TSAN_OPTIONS",
"ASAN_OPTIONS",
"MSAN_OPTIONS",
"UBSAN_OPTIONS",
]:
current_options = os.environ.get(env_name, None)
if current_options is None:
os.environ[env_name] = f"log_path={args.client_log}"
elif "log_path=" not in current_options:
os.environ[env_name] += f":log_path={args.client_log}"
os.environ["CLICKHOUSE_CLIENT_OPT"] = ( os.environ["CLICKHOUSE_CLIENT_OPT"] = (
os.environ["CLICKHOUSE_CLIENT_OPT"] os.environ["CLICKHOUSE_CLIENT_OPT"]
if "CLICKHOUSE_CLIENT_OPT" in os.environ if "CLICKHOUSE_CLIENT_OPT" in os.environ
@ -3109,13 +3122,15 @@ def main(args):
print(colored("\nNo queries hung.", args, "green", attrs=["bold"])) print(colored("\nNo queries hung.", args, "green", attrs=["bold"]))
if args.client_log: if args.client_log:
if os.path.exists(args.client_log): for log_file in [args.client_log, *glob.glob(f"{args.client_log}.*")]:
with open(args.client_log, "rb") as stream: if not os.path.exists(log_file):
continue
with open(log_file, "rb") as stream:
content = stream.read().decode() content = stream.read().decode()
if len(content): if len(content):
print("Has fatal logs from client:\n") print(f"Has fatal logs from client in '{log_file}':\n")
print(content) print(content)
os.remove(args.client_log) os.remove(log_file)
if len(restarted_tests) > 0: if len(restarted_tests) > 0:
print("\nSome tests were restarted:\n") print("\nSome tests were restarted:\n")

View File

@ -7,10 +7,13 @@
:main jepsen.clickhouse.main :main jepsen.clickhouse.main
:plugins [[lein-cljfmt "0.7.0"]] :plugins [[lein-cljfmt "0.7.0"]]
:dependencies [[org.clojure/clojure "1.10.1"] :dependencies [[org.clojure/clojure "1.10.1"]
[jepsen "0.2.7"] [jepsen "0.2.7":exclusions [net.java.dev.jna/jna
net.java.dev.jna/jna-platform]]
[zookeeper-clj "0.9.4"] [zookeeper-clj "0.9.4"]
[org.clojure/java.jdbc "0.7.12"] [org.clojure/java.jdbc "0.7.12"]
[com.hierynomus/sshj "0.34.0"] [com.hierynomus/sshj "0.34.0"]
[net.java.dev.jna/jna "5.14.0"]
[net.java.dev.jna/jna-platform "5.14.0"]
[com.clickhouse/clickhouse-jdbc "0.3.2-patch11"] [com.clickhouse/clickhouse-jdbc "0.3.2-patch11"]
[org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]] [org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]]
:repl-options {:init-ns jepsen.clickhouse-keeper.main} :repl-options {:init-ns jepsen.clickhouse-keeper.main}

View File

@ -40,10 +40,17 @@ OPTIMIZE TABLE shard_0.from_0;
OPTIMIZE TABLE shard_1.from_0; OPTIMIZE TABLE shard_1.from_0;
OPTIMIZE TABLE shard_0.from_1; OPTIMIZE TABLE shard_0.from_1;
OPTIMIZE TABLE shard_1.from_1; OPTIMIZE TABLE shard_1.from_1;
OPTIMIZE TABLE shard_0.to; OPTIMIZE TABLE shard_0.to;
-- If moved parts are not merged by OPTIMIZE or background merge restart
-- can log Warning about metadata version on disk. It's normal situation
-- and test shouldn't rarely fail because of it.
set send_logs_level = 'error';
system restart replica shard_0.to; system restart replica shard_0.to;
-- Doesn't lead to test flakyness, because we don't check anything after it
select sleep(2); select sleep(2);
attach table shard_1.to; attach table shard_1.to;
@ -54,4 +61,3 @@ drop table if exists shard_0.from_1;
drop table if exists shard_1.from_1; drop table if exists shard_1.from_1;
drop table if exists shard_0.to; drop table if exists shard_0.to;
drop table if exists shard_1.to; drop table if exists shard_1.to;

View File

@ -35,8 +35,15 @@ OPTIMIZE TABLE shard_0.to;
OPTIMIZE TABLE shard_0.to; OPTIMIZE TABLE shard_0.to;
select name, active from system.parts where database='shard_0' and table='to' and active order by name; select name, active from system.parts where database='shard_0' and table='to' and active order by name;
-- If moved parts are not merged by OPTIMIZE or background merge restart
-- can log Warning about metadata version on disk. It's normal situation
-- and test shouldn't rarely fail because of it.
set send_logs_level = 'error';
system restart replica shard_0.to; system restart replica shard_0.to;
-- Doesn't lead to test flakyness, because we don't check content in table
-- which doesn't depend on any background operation
select sleep(3); select sleep(3);
attach table shard_1.to; attach table shard_1.to;

View File

@ -0,0 +1,2 @@
0 Value_0
1 Value_1

View File

@ -0,0 +1,23 @@
DROP TABLE IF EXISTS test_table SYNC;
CREATE TABLE test_table
(
id UInt64,
value String
) ENGINE=ReplicatedMergeTree('/clickhouse/test/{database}/test_table', 'r1') ORDER BY tuple();
INSERT INTO test_table VALUES (0, 'Value_0'), (1, 'Value_1'), (2, 'Value_2');
DROP TABLE IF EXISTS test_table_for_in SYNC;
CREATE TABLE test_table_for_in
(
id UInt64
) ENGINE=ReplicatedMergeTree('/clickhouse/test/{database}/test_table_for_in', 'r1') ORDER BY tuple();
INSERT INTO test_table_for_in VALUES (0), (1);
SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost';
SELECT id, value FROM test_table WHERE id IN (SELECT id FROM test_table_for_in UNION DISTINCT SELECT id FROM test_table_for_in);
DROP TABLE test_table SYNC;
DROP TABLE test_table_for_in SYNC;

View File

@ -0,0 +1 @@
CAT 2

View File

@ -0,0 +1,45 @@
DROP TABLE IF EXISTS ANIMAL SYNC;
CREATE TABLE ANIMAL ( ANIMAL Nullable(String) ) ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/animal', 'r1') ORDER BY tuple();
INSERT INTO ANIMAL (ANIMAL) VALUES ('CAT'), ('FISH'), ('DOG'), ('HORSE'), ('BIRD');
SET joined_subquery_requires_alias = 0;
SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost';
SELECT *
FROM
(
SELECT
x.b AS x,
countDistinct(x.c) AS ANIMAL
FROM
(
SELECT
a.ANIMAL AS a,
'CAT' AS b,
c.ANIMAL AS c,
d.ANIMAL AS d
FROM ANIMAL AS a
INNER JOIN ANIMAL AS b ON a.ANIMAL = b.ANIMAL
LEFT JOIN ANIMAL AS c ON b.ANIMAL = c.ANIMAL
RIGHT JOIN
(
SELECT *
FROM ANIMAL
UNION ALL
SELECT *
FROM ANIMAL
UNION ALL
SELECT *
FROM ANIMAL
) AS d ON a.ANIMAL = d.ANIMAL
WHERE (d.ANIMAL != 'CAT') AND (c.ANIMAL != 'DOG') AND (b.ANIMAL != 'FISH')
) AS x
WHERE x.b >= 'CAT'
GROUP BY x.b
HAVING ANIMAL >= 0
) AS ANIMAL
WHERE ANIMAL.ANIMAL >= 0;
DROP TABLE ANIMAL SYNC;

View File

@ -0,0 +1,151 @@
ReadType: InReverseOrder
2024-06-11 02:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-11 00:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 22:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 20:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 18:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 16:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 14:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 12:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 10:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 08:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 06:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 04:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 02:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 02:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 00:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-10 00:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 22:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 22:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 20:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 20:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 18:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 18:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 16:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 16:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 14:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 14:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 12:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 12:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 10:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 10:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 08:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 08:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 06:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 06:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 04:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 04:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 02:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 02:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 02:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 00:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 00:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-09 00:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 22:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 22:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 22:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 20:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 20:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 20:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 18:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 18:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 18:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 16:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 16:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 16:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 14:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 14:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 14:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 12:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 12:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 12:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 10:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 10:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 10:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 08:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 08:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 08:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 06:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 06:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 06:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 04:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 04:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 04:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 02:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 02:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 02:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 00:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 00:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-08 00:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 22:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 22:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 22:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 20:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 20:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 20:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 18:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 18:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 18:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 16:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 16:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 16:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 14:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 14:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 14:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 12:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 12:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 12:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 10:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 10:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 10:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 08:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 08:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 08:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 06:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 06:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 06:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 04:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 04:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 04:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 02:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 02:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 02:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 00:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 00:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-07 00:00:01 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 22:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 22:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 20:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 20:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 18:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 18:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 16:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 16:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 14:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 14:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 12:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 12:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 10:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 10:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 08:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 08:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 06:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 06:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 04:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 04:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 02:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 02:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 00:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-06 00:00:02 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 22:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 20:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 18:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 16:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 14:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 12:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 10:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 08:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 06:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 04:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 02:00:03 asan 02221_parallel_replicas_bug FAIL fail
2024-06-05 00:00:03 asan 02221_parallel_replicas_bug FAIL fail

View File

@ -0,0 +1,49 @@
DROP TABLE IF EXISTS checks SYNC;
CREATE TABLE checks
(
`check_name` LowCardinality(String),
`check_status` LowCardinality(String),
`check_start_time` DateTime,
`test_name` LowCardinality(String),
`test_status` LowCardinality(String),
)
ENGINE = ReplicatedMergeTree('/clickhouse/{database}/checks', '{replica}')
ORDER BY check_start_time;
SYSTEM STOP MERGES checks;
INSERT INTO checks SELECT 'asan', if(number % 2, 'success', 'fail'), toDateTime('2024-06-07 00:00:01') + INTERVAL number HOUR, '02221_parallel_replicas_bug', 'FAIL' from numbers(100);
INSERT INTO checks SELECT 'asan', if(number % 2, 'success', 'fail'), toDateTime('2024-06-06 00:00:02') + INTERVAL number HOUR, '02221_parallel_replicas_bug', 'FAIL' from numbers(100);
INSERT INTO checks SELECT 'asan', if(number % 2, 'success', 'fail'), toDateTime('2024-06-05 00:00:03') + INTERVAL number HOUR, '02221_parallel_replicas_bug', 'FAIL' from numbers(100);
SELECT trimBoth(explain)
FROM
(
EXPLAIN actions=1 SELECT
check_start_time,
check_name,
test_name,
test_status,
check_status
FROM checks
WHERE 1 AND (test_status != 'SKIPPED') AND (test_status != 'OK') AND (check_status != 'success') AND (test_name ILIKE '%parallel_replicas%')
ORDER BY
check_start_time DESC,
check_name ASC,
test_name ASC
SETTINGS query_plan_read_in_order = 1, optimize_read_in_order = 1, max_parallel_replicas = 1
)
WHERE explain LIKE '%InReverseOrder%';
SELECT check_start_time, check_name, test_name, test_status, check_status
FROM checks
WHERE 1
AND test_status != 'SKIPPED'
AND test_status != 'OK'
AND check_status != 'success'
AND test_name ilike '%parallel_replicas%'
ORDER BY check_start_time desc, check_name, test_name
SETTINGS query_plan_read_in_order = 1, optimize_read_in_order = 1, allow_experimental_parallel_reading_from_replicas = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3;
DROP TABLE checks SYNC;