Skip to content

Commit 47023ac

Browse files
authored
Merge branch 'master' into delta-kernel-rs-support-local-storage
2 parents cb9e79e + 26c958f commit 47023ac

File tree

99 files changed

+1029
-415
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

99 files changed

+1029
-415
lines changed

ci/defs/job_configs.py

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -867,9 +867,13 @@ class JobConfigs:
867867
runs_on=RunnerLabels.FUNC_TESTER_AMD,
868868
command="python3 ./ci/jobs/clickbench.py",
869869
digest_config=Job.CacheDigestConfig(
870-
include_paths=["./ci/jobs/clickbench.py", "./ci/jobs/scripts/clickbench/"],
870+
include_paths=[
871+
"./ci/jobs/clickbench.py",
872+
"./ci/jobs/scripts/clickbench/",
873+
"./ci/jobs/scripts/functional_tests/setup_log_cluster.sh",
874+
],
871875
),
872-
run_in_docker="clickhouse/stateless-test+--shm-size=16g",
876+
run_in_docker="clickhouse/stateless-test+--shm-size=16g+--network=host",
873877
).parametrize(
874878
parameter=[
875879
BuildTypes.AMD_RELEASE,

ci/jobs/build_clickhouse.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,7 @@
1313
temp_dir = f"{current_directory}/ci/tmp"
1414

1515
BUILD_TYPE_TO_CMAKE = {
16-
BuildTypes.AMD_DEBUG: f" cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=Debug -DENABLE_THINLTO=0 -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DCMAKE_C_COMPILER={ToolSet.COMPILER_C} -DCMAKE_CXX_COMPILER={ToolSet.COMPILER_CPP} -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 -DENABLE_TESTS=1 -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON",
16+
BuildTypes.AMD_DEBUG: f" cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=Debug -DENABLE_THINLTO=0 -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DCMAKE_C_COMPILER={ToolSet.COMPILER_C} -DCMAKE_CXX_COMPILER={ToolSet.COMPILER_CPP} -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 -DENABLE_TESTS=1 -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON -DENABLE_XRAY=1",
1717
BuildTypes.AMD_RELEASE: f" cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=None -DENABLE_THINLTO=1 -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DCMAKE_C_COMPILER={ToolSet.COMPILER_C} -DCMAKE_CXX_COMPILER={ToolSet.COMPILER_CPP} -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 -DENABLE_TESTS=0 -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON -DSPLIT_DEBUG_SYMBOLS=ON -DBUILD_STANDALONE_KEEPER=1",
1818
BuildTypes.AMD_BINARY: f" cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=None -DENABLE_THINLTO=0 -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DCMAKE_C_COMPILER={ToolSet.COMPILER_C} -DCMAKE_CXX_COMPILER={ToolSet.COMPILER_CPP} -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 -DENABLE_TESTS=0 -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON",
1919
BuildTypes.AMD_ASAN: f" cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=None -DENABLE_THINLTO=0 -DSANITIZE=address -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DCMAKE_C_COMPILER={ToolSet.COMPILER_C} -DCMAKE_CXX_COMPILER={ToolSet.COMPILER_CPP} -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 -DENABLE_TESTS=1 -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON",

ci/jobs/clickbench.py

Lines changed: 17 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
1-
from praktika.result import Result
2-
from praktika.utils import Shell, Utils
3-
41
from ci.jobs.scripts.clickhouse_proc import ClickHouseLight
2+
from ci.praktika.info import Info
3+
from ci.praktika.result import Result
4+
from ci.praktika.utils import Shell, Utils
55

66
temp_dir = f"{Utils.cwd()}/ci/tmp/"
77

@@ -16,7 +16,11 @@ def main():
1616
print("Install ClickHouse")
1717

1818
def install():
19-
return ch.install() and ch.clickbench_config_tweaks()
19+
return (
20+
ch.install()
21+
and ch.clickbench_config_tweaks()
22+
and ch.create_log_export_config()
23+
)
2024

2125
results.append(
2226
Result.from_commands_run(name="Install ClickHouse", command=install)
@@ -27,15 +31,16 @@ def install():
2731
print("Start ClickHouse")
2832

2933
def start():
30-
return ch.start()
31-
32-
log_export_config = f"./ci/jobs/scripts/functional_tests/setup_log_cluster.sh --config-logs-export-cluster {ch.config_path}/config.d/system_logs_export.yaml"
33-
setup_logs_replication = f"./ci/jobs/scripts/functional_tests/setup_log_cluster.sh --setup-logs-replication"
34+
return ch.start() and (
35+
ch.start_log_exports(check_start_time=stop_watch.start_time)
36+
if not Info().is_local_run
37+
else True
38+
)
3439

3540
results.append(
3641
Result.from_commands_run(
3742
name="Start ClickHouse",
38-
command=[start, log_export_config, setup_logs_replication],
43+
command=start,
3944
)
4045
)
4146
res = results[-1].is_ok()
@@ -92,7 +97,9 @@ def start():
9297
verbose=True,
9398
)
9499

95-
Result.create_from(results=results, stopwatch=stop_watch, files=[]).complete_job()
100+
Result.create_from(
101+
results=results, stopwatch=stop_watch, files=[ch.log_file]
102+
).complete_job()
96103

97104

98105
if __name__ == "__main__":

ci/jobs/scripts/clickhouse_proc.py

Lines changed: 61 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3,11 +3,28 @@
33
import time
44
from pathlib import Path
55

6-
from praktika.utils import Shell, Utils
6+
from ci.praktika import Secret
7+
from ci.praktika.info import Info
8+
from ci.praktika.utils import Shell, Utils
79

810
temp_dir = f"{Utils.cwd()}/ci/tmp"
911

1012

13+
LOG_EXPORT_CONFIG_TEMPLATE = """
14+
remote_servers:
15+
{CLICKHOUSE_CI_LOGS_CLUSTER}:
16+
shard:
17+
replica:
18+
secure: 1
19+
user: '{CLICKHOUSE_CI_LOGS_USER}'
20+
host: '{CLICKHOUSE_CI_LOGS_HOST}'
21+
port: 9440
22+
password: '{CLICKHOUSE_CI_LOGS_PASSWORD}'
23+
"""
24+
CLICKHOUSE_CI_LOGS_CLUSTER = "system_logs_export"
25+
CLICKHOUSE_CI_LOGS_USER = "ci"
26+
27+
1128
class ClickHouseProc:
1229
BACKUPS_XML = """
1330
<clickhouse>
@@ -232,6 +249,49 @@ def fuzzer_config_tweaks(self):
232249
res = res and Shell.check(command, verbose=True)
233250
return res
234251

252+
def create_log_export_config(self):
253+
print("Create log export config")
254+
config_file = Path(self.config_path) / "config.d" / "system_logs_export.yaml"
255+
256+
self.log_export_host = Secret.Config(
257+
name="clickhouse_ci_logs_host",
258+
type=Secret.Type.AWS_SSM_VAR,
259+
region="us-east-1",
260+
).get_value()
261+
262+
self.log_export_password = Secret.Config(
263+
name="clickhouse_ci_logs_password",
264+
type=Secret.Type.AWS_SSM_VAR,
265+
region="us-east-1",
266+
).get_value()
267+
268+
config_content = LOG_EXPORT_CONFIG_TEMPLATE.format(
269+
CLICKHOUSE_CI_LOGS_CLUSTER=CLICKHOUSE_CI_LOGS_CLUSTER,
270+
CLICKHOUSE_CI_LOGS_HOST=self.log_export_host,
271+
CLICKHOUSE_CI_LOGS_USER=CLICKHOUSE_CI_LOGS_USER,
272+
CLICKHOUSE_CI_LOGS_PASSWORD=self.log_export_password,
273+
)
274+
275+
with open(config_file, "w") as f:
276+
f.write(config_content)
277+
278+
def start_log_exports(self, check_start_time):
279+
print("Start log export")
280+
os.environ["CLICKHOUSE_CI_LOGS_CLUSTER"] = CLICKHOUSE_CI_LOGS_CLUSTER
281+
os.environ["CLICKHOUSE_CI_LOGS_HOST"] = self.log_export_host
282+
os.environ["CLICKHOUSE_CI_LOGS_USER"] = CLICKHOUSE_CI_LOGS_USER
283+
os.environ["CLICKHOUSE_CI_LOGS_PASSWORD"] = self.log_export_password
284+
info = Info()
285+
os.environ["EXTRA_COLUMNS_EXPRESSION"] = (
286+
f"CAST({info.pr_number} AS UInt32) AS pull_request_number, '{info.sha}' AS commit_sha, toDateTime('{Utils.timestamp_to_str(check_start_time)}', 'UTC') AS check_start_time, toLowCardinality('{info.job_name}') AS check_name, toLowCardinality('{info.instance_type}') AS instance_type, '{info.instance_id}' AS instance_id"
287+
)
288+
289+
Shell.check(
290+
"./ci/jobs/scripts/functional_tests/setup_log_cluster.sh --setup-logs-replication",
291+
verbose=True,
292+
strict=True,
293+
)
294+
235295
def start(self):
236296
print(f"Starting ClickHouse server")
237297
print("Command: ", self.start_cmd)

ci/jobs/scripts/functional_tests/setup_log_cluster.sh

Lines changed: 6 additions & 49 deletions
Original file line numberDiff line numberDiff line change
@@ -9,15 +9,15 @@ set -e
99
# Config file contains KEY=VALUE pairs with any necessary parameters like:
1010
# CLICKHOUSE_CI_LOGS_HOST - remote host
1111
# CLICKHOUSE_CI_LOGS_USER - password for user
12-
# CLICKHOUSE_CI_LOGS_PASSWORD - password for user
13-
CLICKHOUSE_CI_LOGS_CREDENTIALS=${CLICKHOUSE_CI_LOGS_CREDENTIALS:-/tmp/export-logs-config.sh}
14-
CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci}
12+
# CLICKHOUSE_CI_LOGS_PACreate all configured system logsSSWORD - password for user
1513

1614
# Pre-configured destination cluster, where to export the data
1715
CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export}
1816

17+
[ -n "$EXTRA_COLUMNS_EXPRESSION" ] || { echo "ERROR: EXTRA_COLUMNS_EXPRESSION env must be defined"; exit 1; }
1918
EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, INDEX ix_pr (pull_request_number) TYPE set(100), INDEX ix_commit (commit_sha) TYPE set(100), INDEX ix_check_time (check_start_time) TYPE minmax, "}
2019
EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id"}
20+
echo "EXTRA_COLUMNS_EXPRESSION=$EXTRA_COLUMNS_EXPRESSION"
2121
EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name"}
2222

2323
# trace_log needs more columns for symbolization
@@ -57,6 +57,7 @@ function check_logs_credentials
5757

5858
# First check, if all necessary parameters are set
5959
set +x
60+
echo "Check CI Log cluster..."
6061
for parameter in CLICKHOUSE_CI_LOGS_HOST CLICKHOUSE_CI_LOGS_USER CLICKHOUSE_CI_LOGS_PASSWORD; do
6162
export -p | grep -q "$parameter" || {
6263
echo "Credentials parameter $parameter is unset"
@@ -75,51 +76,12 @@ function check_logs_credentials
7576
fi
7677
)
7778

78-
function config_logs_export_cluster
79-
(
80-
# The function is launched in a separate shell instance to not expose the
81-
# exported values from CLICKHOUSE_CI_LOGS_CREDENTIALS
82-
set +x
83-
if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then
84-
echo "File $CLICKHOUSE_CI_LOGS_CREDENTIALS does not exist, do not setup"
85-
return
86-
fi
87-
set -a
88-
# shellcheck disable=SC1090
89-
source "${CLICKHOUSE_CI_LOGS_CREDENTIALS}"
90-
set +a
91-
__shadow_credentials
92-
echo "Checking if the credentials work"
93-
check_logs_credentials || return 0
94-
cluster_config="${1:-/etc/clickhouse-server/config.d/system_logs_export.yaml}"
95-
mkdir -p "$(dirname "$cluster_config")"
96-
echo "remote_servers:
97-
${CLICKHOUSE_CI_LOGS_CLUSTER}:
98-
shard:
99-
replica:
100-
secure: 1
101-
user: '${CLICKHOUSE_CI_LOGS_USER}'
102-
host: '${CLICKHOUSE_CI_LOGS_HOST}'
103-
port: 9440
104-
password: '${CLICKHOUSE_CI_LOGS_PASSWORD}'
105-
" > "$cluster_config"
106-
echo "Cluster ${CLICKHOUSE_CI_LOGS_CLUSTER} is confugured in ${cluster_config}"
107-
)
108-
10979
function setup_logs_replication
11080
(
11181
# The function is launched in a separate shell instance to not expose the
112-
# exported values from CLICKHOUSE_CI_LOGS_CREDENTIALS
82+
# exported values
11383
set +x
11484
# disable output
115-
if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then
116-
echo "File $CLICKHOUSE_CI_LOGS_CREDENTIALS does not exist, do not setup"
117-
return 0
118-
fi
119-
set -a
120-
# shellcheck disable=SC1090
121-
source "${CLICKHOUSE_CI_LOGS_CREDENTIALS}"
122-
set +a
12385
__shadow_credentials
12486
echo "Checking if the credentials work"
12587
check_logs_credentials || return 0
@@ -246,14 +208,9 @@ while [[ "$#" -gt 0 ]]; do
246208
echo "Setting up log replication..."
247209
setup_logs_replication
248210
;;
249-
--config-logs-export-cluster)
250-
echo "Configuring logs export for the cluster..."
251-
config_logs_export_cluster "$2"
252-
shift
253-
;;
254211
*)
255212
echo "Unknown option: $1"
256-
echo "Usage: $0 [--stop-log-replication | --setup-logs-replication | --config-logs-export-cluster ]"
213+
echo "Usage: $0 [--stop-log-replication | --setup-logs-replication ]"
257214
exit 1
258215
;;
259216
esac

ci/praktika/secret.py

Lines changed: 10 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ class Type:
1515
class Config:
1616
name: str
1717
type: str
18+
region: str = ""
1819

1920
def is_gh_secret(self):
2021
return self.type == Secret.Type.GH_SECRET
@@ -33,8 +34,11 @@ def get_value(self):
3334
assert False, f"Not supported secret type, secret [{self}]"
3435

3536
def get_aws_ssm_var(self):
37+
region = ""
38+
if self.region:
39+
region = f" --region {self.region}"
3640
res = Shell.get_output(
37-
f"aws ssm get-parameter --name {self.name} --with-decryption --output text --query Parameter.Value",
41+
f"aws ssm get-parameter --name {self.name} --with-decryption --output text --query Parameter.Value {region}",
3842
strict=True,
3943
)
4044
return res
@@ -43,13 +47,13 @@ def get_aws_ssm_secret(self):
4347
name, secret_key_name = self.name, ""
4448
if "." in self.name:
4549
name, secret_key_name = self.name.split(".")
46-
cmd = f"aws secretsmanager get-secret-value --secret-id {name} --query SecretString --output text"
50+
region = ""
51+
if self.region:
52+
region = f" --region {self.region}"
53+
cmd = f"aws secretsmanager get-secret-value --secret-id {name} --query SecretString --output text {region}"
4754
if secret_key_name:
4855
cmd += f" | jq -r '.[\"{secret_key_name}\"]'"
49-
res = Shell.get_output(cmd, verbose=True)
50-
if not res:
51-
print(f"ERROR: Failed to get secret [{self.name}]")
52-
raise RuntimeError()
56+
res = Shell.get_output(cmd, verbose=True, strict=True)
5357
return res
5458

5559
def get_gh_secret(self):

src/Backups/BackupCoordinationReplicatedTables.cpp

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ class BackupCoordinationReplicatedTables::CoveredPartsFinder
3636
{
3737
auto new_min_block = new_part_info.min_block;
3838
auto new_max_block = new_part_info.max_block;
39-
auto & parts = partitions[new_part_info.partition_id];
39+
auto & parts = partitions[new_part_info.getPartitionId()];
4040

4141
/// Find the first part with max_block >= `part_info.min_block`.
4242
auto first_it = parts.lower_bound(new_min_block);
@@ -99,7 +99,7 @@ class BackupCoordinationReplicatedTables::CoveredPartsFinder
9999

100100
bool isCoveredByAnotherPart(const MergeTreePartInfo & part_info) const
101101
{
102-
auto partition_it = partitions.find(part_info.partition_id);
102+
auto partition_it = partitions.find(part_info.getPartitionId());
103103
if (partition_it == partitions.end())
104104
return false;
105105

@@ -290,11 +290,12 @@ void BackupCoordinationReplicatedTables::prepare() const
290290
for (const auto & [part_name, part_replicas] : table_info.replicas_by_part_name)
291291
{
292292
auto part_info = MergeTreePartInfo::fromPartName(part_name, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING);
293+
const auto & partition_id = part_info.getPartitionId();
293294

294295
auto & min_data_versions_by_partition = table_info.min_data_versions_by_partition;
295-
auto it2 = min_data_versions_by_partition.find(part_info.partition_id);
296+
auto it2 = min_data_versions_by_partition.find(partition_id);
296297
if (it2 == min_data_versions_by_partition.end())
297-
min_data_versions_by_partition[part_info.partition_id] = part_info.getDataVersion();
298+
min_data_versions_by_partition[partition_id] = part_info.getDataVersion();
298299
else
299300
it2->second = std::min(it2->second, part_info.getDataVersion());
300301

src/Backups/RestorerFromBackup.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -485,7 +485,7 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_
485485
res_table_info.has_data = table_has_data;
486486
res_table_info.data_path_in_backup = data_path_in_backup;
487487

488-
tables_dependencies.addDependencies(table_name, table_dependencies);
488+
tables_dependencies.addDependencies(table_name, table_dependencies.dependencies);
489489

490490
if (partitions)
491491
{

src/Core/ServerSettings.cpp

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1057,6 +1057,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_
10571057
```
10581058
)", 0) \
10591059
DECLARE(Bool, storage_shared_set_join_use_inner_uuid, true, "If enabled, an inner UUID is generated during the creation of SharedSet and SharedJoin. ClickHouse Cloud only", 0) \
1060+
DECLARE(UInt64, startup_mv_delay_ms, 0, R"(Debug parameter to simulate materizlied view creation delay)", 0) \
10601061
DECLARE(UInt64, os_cpu_busy_time_threshold, 1'000'000, "Threshold of OS CPU busy time in microseconds (OSCPUVirtualTimeMicroseconds metric) to consider CPU doing some useful work, no CPU overload would be considered if busy time was below this value.", 0) \
10611062
DECLARE(Float, min_os_cpu_wait_time_ratio_to_drop_connection, 10.0, "Min ratio between OS CPU wait (OSCPUWaitMicroseconds metric) and busy (OSCPUVirtualTimeMicroseconds metric) times to consider dropping connections. Linear interpolation between min and max ratio is used to calculate the probability, the probability is 0 at this point.", 0) \
10621063
DECLARE(Float, max_os_cpu_wait_time_ratio_to_drop_connection, 20.0, "Max ratio between OS CPU wait (OSCPUWaitMicroseconds metric) and busy (OSCPUVirtualTimeMicroseconds metric) times to consider dropping connections. Linear interpolation between min and max ratio is used to calculate the probability, the probability is 1 at this point.", 0) \

0 commit comments

Comments
 (0)