Skip to content

Commit 7246911

Browse files
authored
Merge branch 'releases/25.8.13' into builds/stable/fixed-version
2 parents b51d6bc + db73ca5 commit 7246911

Some content is hidden

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

47 files changed

+330
-174
lines changed

.github/workflows/pull_request.yml

Lines changed: 1 addition & 93 deletions
Original file line numberDiff line numberDiff line change
@@ -2166,96 +2166,6 @@ jobs:
21662166
python3 -m praktika run 'Stateless tests (arm_binary, sequential)' --workflow "PR" --ci |& tee ./ci/tmp/job.log
21672167
fi
21682168
2169-
bugfix_validation_integration_tests:
2170-
runs-on: [self-hosted, altinity-on-demand, altinity-func-tester]
2171-
needs: [config_workflow, dockers_build_amd, dockers_build_arm, dockers_build_multiplatform_manifest]
2172-
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVnZml4IHZhbGlkYXRpb24gKGludGVncmF0aW9uIHRlc3RzKQ==') }}
2173-
name: "Bugfix validation (integration tests)"
2174-
outputs:
2175-
data: ${{ steps.run.outputs.DATA }}
2176-
steps:
2177-
- name: Checkout code
2178-
uses: actions/checkout@v4
2179-
with:
2180-
ref: ${{ env.CHECKOUT_REF }}
2181-
2182-
- name: Setup
2183-
uses: ./.github/actions/runner_setup
2184-
- name: Docker setup
2185-
uses: ./.github/actions/docker_setup
2186-
with:
2187-
test_name: "Bugfix validation (integration tests)"
2188-
2189-
- name: Prepare env script
2190-
run: |
2191-
rm -rf ./ci/tmp ./ci/tmp ./ci/tmp
2192-
mkdir -p ./ci/tmp ./ci/tmp ./ci/tmp
2193-
cat > ./ci/tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF'
2194-
export PYTHONPATH=./ci:.:
2195-
cat > ./ci/tmp/workflow_config_pr.json << 'EOF'
2196-
${{ needs.config_workflow.outputs.data }}
2197-
EOF
2198-
cat > ./ci/tmp/workflow_status.json << 'EOF'
2199-
${{ toJson(needs) }}
2200-
EOF
2201-
ENV_SETUP_SCRIPT_EOF
2202-
2203-
- name: Run
2204-
id: run
2205-
run: |
2206-
. ./ci/tmp/praktika_setup_env.sh
2207-
set -o pipefail
2208-
if command -v ts &> /dev/null; then
2209-
python3 -m praktika run 'Bugfix validation (integration tests)' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee ./ci/tmp/job.log
2210-
else
2211-
python3 -m praktika run 'Bugfix validation (integration tests)' --workflow "PR" --ci |& tee ./ci/tmp/job.log
2212-
fi
2213-
2214-
bugfix_validation_functional_tests:
2215-
runs-on: [self-hosted, altinity-on-demand, altinity-func-tester-aarch64]
2216-
needs: [config_workflow, dockers_build_amd, dockers_build_arm, dockers_build_multiplatform_manifest]
2217-
if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVnZml4IHZhbGlkYXRpb24gKGZ1bmN0aW9uYWwgdGVzdHMp') }}
2218-
name: "Bugfix validation (functional tests)"
2219-
outputs:
2220-
data: ${{ steps.run.outputs.DATA }}
2221-
steps:
2222-
- name: Checkout code
2223-
uses: actions/checkout@v4
2224-
with:
2225-
ref: ${{ env.CHECKOUT_REF }}
2226-
2227-
- name: Setup
2228-
uses: ./.github/actions/runner_setup
2229-
- name: Docker setup
2230-
uses: ./.github/actions/docker_setup
2231-
with:
2232-
test_name: "Bugfix validation (functional tests)"
2233-
2234-
- name: Prepare env script
2235-
run: |
2236-
rm -rf ./ci/tmp ./ci/tmp ./ci/tmp
2237-
mkdir -p ./ci/tmp ./ci/tmp ./ci/tmp
2238-
cat > ./ci/tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF'
2239-
export PYTHONPATH=./ci:.:
2240-
cat > ./ci/tmp/workflow_config_pr.json << 'EOF'
2241-
${{ needs.config_workflow.outputs.data }}
2242-
EOF
2243-
cat > ./ci/tmp/workflow_status.json << 'EOF'
2244-
${{ toJson(needs) }}
2245-
EOF
2246-
ENV_SETUP_SCRIPT_EOF
2247-
2248-
- name: Run
2249-
id: run
2250-
run: |
2251-
. ./ci/tmp/praktika_setup_env.sh
2252-
set -o pipefail
2253-
if command -v ts &> /dev/null; then
2254-
python3 -m praktika run 'Bugfix validation (functional tests)' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee ./ci/tmp/job.log
2255-
else
2256-
python3 -m praktika run 'Bugfix validation (functional tests)' --workflow "PR" --ci |& tee ./ci/tmp/job.log
2257-
fi
2258-
22592169
stateless_tests_amd_asan_flaky_check:
22602170
runs-on: [self-hosted, altinity-on-demand, altinity-style-checker]
22612171
needs: [config_workflow, dockers_build_amd, dockers_build_arm, dockers_build_multiplatform_manifest, build_amd_asan]
@@ -4058,7 +3968,7 @@ jobs:
40583968
40593969
finish_workflow:
40603970
runs-on: [self-hosted, altinity-on-demand, altinity-style-checker-aarch64]
4061-
needs: [config_workflow, dockers_build_amd, dockers_build_arm, dockers_build_multiplatform_manifest, fast_test, build_amd_debug, build_amd_release, build_amd_asan, build_amd_tsan, build_amd_msan, build_amd_ubsan, build_amd_binary, build_arm_release, build_arm_coverage, build_arm_binary, unit_tests_asan, unit_tests_tsan, unit_tests_msan, unit_tests_ubsan, stateless_tests_amd_asan_distributed_plan_parallel_1_2, stateless_tests_amd_asan_distributed_plan_parallel_2_2, stateless_tests_amd_asan_distributed_plan_sequential, stateless_tests_amd_binary_old_analyzer_s3_storage_databasereplicated_parallel, stateless_tests_amd_binary_old_analyzer_s3_storage_databasereplicated_sequential, stateless_tests_amd_binary_parallelreplicas_s3_storage_parallel, stateless_tests_amd_binary_parallelreplicas_s3_storage_sequential, stateless_tests_amd_debug_asyncinsert_s3_storage_parallel, stateless_tests_amd_debug_asyncinsert_s3_storage_sequential, stateless_tests_amd_debug_parallel, stateless_tests_amd_debug_sequential, stateless_tests_amd_tsan_parallel_1_2, stateless_tests_amd_tsan_parallel_2_2, stateless_tests_amd_tsan_sequential_1_2, stateless_tests_amd_tsan_sequential_2_2, stateless_tests_amd_msan_parallel_1_2, stateless_tests_amd_msan_parallel_2_2, stateless_tests_amd_msan_sequential_1_2, stateless_tests_amd_msan_sequential_2_2, stateless_tests_amd_ubsan_parallel, stateless_tests_amd_ubsan_sequential, stateless_tests_amd_debug_distributed_plan_s3_storage_parallel, stateless_tests_amd_debug_distributed_plan_s3_storage_sequential, stateless_tests_amd_tsan_s3_storage_parallel, stateless_tests_amd_tsan_s3_storage_sequential_1_2, stateless_tests_amd_tsan_s3_storage_sequential_2_2, stateless_tests_arm_binary_parallel, stateless_tests_arm_binary_sequential, bugfix_validation_integration_tests, bugfix_validation_functional_tests, stateless_tests_amd_asan_flaky_check, integration_tests_amd_asan_old_analyzer_1_6, integration_tests_amd_asan_old_analyzer_2_6, integration_tests_amd_asan_old_analyzer_3_6, integration_tests_amd_asan_old_analyzer_4_6, integration_tests_amd_asan_old_analyzer_5_6, integration_tests_amd_asan_old_analyzer_6_6, integration_tests_amd_binary_1_5, integration_tests_amd_binary_2_5, integration_tests_amd_binary_3_5, integration_tests_amd_binary_4_5, integration_tests_amd_binary_5_5, integration_tests_arm_binary_distributed_plan_1_4, integration_tests_arm_binary_distributed_plan_2_4, integration_tests_arm_binary_distributed_plan_3_4, integration_tests_arm_binary_distributed_plan_4_4, integration_tests_amd_tsan_1_6, integration_tests_amd_tsan_2_6, integration_tests_amd_tsan_3_6, integration_tests_amd_tsan_4_6, integration_tests_amd_tsan_5_6, integration_tests_amd_tsan_6_6, integration_tests_amd_asan_flaky_check, docker_server_image, docker_keeper_image, install_packages_amd_debug, compatibility_check_release, compatibility_check_aarch64, stress_test_amd_debug, stress_test_amd_tsan, stress_test_amd_ubsan, stress_test_amd_msan, ast_fuzzer_amd_debug, ast_fuzzer_amd_tsan, ast_fuzzer_amd_msan, ast_fuzzer_amd_ubsan, buzzhouse_amd_debug, buzzhouse_amd_tsan, buzzhouse_amd_msan, buzzhouse_amd_ubsan]
3971+
needs: [config_workflow, dockers_build_amd, dockers_build_arm, dockers_build_multiplatform_manifest, fast_test, build_amd_debug, build_amd_release, build_amd_asan, build_amd_tsan, build_amd_msan, build_amd_ubsan, build_amd_binary, build_arm_release, build_arm_coverage, build_arm_binary, unit_tests_asan, unit_tests_tsan, unit_tests_msan, unit_tests_ubsan, stateless_tests_amd_asan_distributed_plan_parallel_1_2, stateless_tests_amd_asan_distributed_plan_parallel_2_2, stateless_tests_amd_asan_distributed_plan_sequential, stateless_tests_amd_binary_old_analyzer_s3_storage_databasereplicated_parallel, stateless_tests_amd_binary_old_analyzer_s3_storage_databasereplicated_sequential, stateless_tests_amd_binary_parallelreplicas_s3_storage_parallel, stateless_tests_amd_binary_parallelreplicas_s3_storage_sequential, stateless_tests_amd_debug_asyncinsert_s3_storage_parallel, stateless_tests_amd_debug_asyncinsert_s3_storage_sequential, stateless_tests_amd_debug_parallel, stateless_tests_amd_debug_sequential, stateless_tests_amd_tsan_parallel_1_2, stateless_tests_amd_tsan_parallel_2_2, stateless_tests_amd_tsan_sequential_1_2, stateless_tests_amd_tsan_sequential_2_2, stateless_tests_amd_msan_parallel_1_2, stateless_tests_amd_msan_parallel_2_2, stateless_tests_amd_msan_sequential_1_2, stateless_tests_amd_msan_sequential_2_2, stateless_tests_amd_ubsan_parallel, stateless_tests_amd_ubsan_sequential, stateless_tests_amd_debug_distributed_plan_s3_storage_parallel, stateless_tests_amd_debug_distributed_plan_s3_storage_sequential, stateless_tests_amd_tsan_s3_storage_parallel, stateless_tests_amd_tsan_s3_storage_sequential_1_2, stateless_tests_amd_tsan_s3_storage_sequential_2_2, stateless_tests_arm_binary_parallel, stateless_tests_arm_binary_sequential, stateless_tests_amd_asan_flaky_check, integration_tests_amd_asan_old_analyzer_1_6, integration_tests_amd_asan_old_analyzer_2_6, integration_tests_amd_asan_old_analyzer_3_6, integration_tests_amd_asan_old_analyzer_4_6, integration_tests_amd_asan_old_analyzer_5_6, integration_tests_amd_asan_old_analyzer_6_6, integration_tests_amd_binary_1_5, integration_tests_amd_binary_2_5, integration_tests_amd_binary_3_5, integration_tests_amd_binary_4_5, integration_tests_amd_binary_5_5, integration_tests_arm_binary_distributed_plan_1_4, integration_tests_arm_binary_distributed_plan_2_4, integration_tests_arm_binary_distributed_plan_3_4, integration_tests_arm_binary_distributed_plan_4_4, integration_tests_amd_tsan_1_6, integration_tests_amd_tsan_2_6, integration_tests_amd_tsan_3_6, integration_tests_amd_tsan_4_6, integration_tests_amd_tsan_5_6, integration_tests_amd_tsan_6_6, integration_tests_amd_asan_flaky_check, docker_server_image, docker_keeper_image, install_packages_amd_debug, compatibility_check_release, compatibility_check_aarch64, stress_test_amd_debug, stress_test_amd_tsan, stress_test_amd_ubsan, stress_test_amd_msan, ast_fuzzer_amd_debug, ast_fuzzer_amd_tsan, ast_fuzzer_amd_msan, ast_fuzzer_amd_ubsan, buzzhouse_amd_debug, buzzhouse_amd_tsan, buzzhouse_amd_msan, buzzhouse_amd_ubsan]
40623972
if: ${{ !cancelled() }}
40633973
name: "Finish Workflow"
40643974
outputs:
@@ -4202,8 +4112,6 @@ jobs:
42024112
- stateless_tests_amd_tsan_s3_storage_sequential_2_2
42034113
- stateless_tests_arm_binary_parallel
42044114
- stateless_tests_arm_binary_sequential
4205-
- bugfix_validation_integration_tests
4206-
- bugfix_validation_functional_tests
42074115
- stateless_tests_amd_asan_flaky_check
42084116
- integration_tests_amd_asan_old_analyzer_1_6
42094117
- integration_tests_amd_asan_old_analyzer_2_6

ci/jobs/scripts/clickhouse_proc.py

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -131,14 +131,16 @@ def start_minio(self, test_type):
131131
)
132132
print(f"Started setup_minio.sh asynchronously with PID {self.minio_proc.pid}")
133133

134-
for _ in range(20):
134+
print("Waiting for minio to start...")
135+
for _ in range(10):
135136
res = Shell.check(
136137
"/mc ls clickminio/test | grep -q .",
137138
verbose=True,
138139
)
139140
if res:
141+
print("Minio started successfully")
140142
return True
141-
time.sleep(1)
143+
time.sleep(3)
142144
print("Failed to start minio")
143145
return False
144146

ci/workflows/pull_request.py

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -53,14 +53,14 @@
5353
)
5454
for j in JobConfigs.functional_tests_jobs
5555
],
56-
JobConfigs.bugfix_validation_it_job.set_dependency(
57-
[
58-
# JobNames.STYLE_CHECK, # NOTE (strtgbb): we don't run style check
59-
# JobNames.FAST_TEST, # NOTE (strtgbb): we don't run fast tests
60-
# JobConfigs.tidy_build_arm_jobs[0].name, # NOTE (strtgbb): we don't run tidy build jobs
61-
]
62-
),
63-
JobConfigs.bugfix_validation_ft_pr_job,
56+
# JobConfigs.bugfix_validation_it_job.set_dependency( # NOTE (strtgbb): depends on upstream build
57+
# [
58+
# # JobNames.STYLE_CHECK, # NOTE (strtgbb): we don't run style check
59+
# # JobNames.FAST_TEST, # NOTE (strtgbb): we don't run fast tests
60+
# # JobConfigs.tidy_build_arm_jobs[0].name, # NOTE (strtgbb): we don't run tidy build jobs
61+
# ]
62+
# ),
63+
# JobConfigs.bugfix_validation_ft_pr_job,
6464
*JobConfigs.stateless_tests_flaky_pr_jobs,
6565
*[
6666
job.set_dependency(FUNCTIONAL_TESTS_PARALLEL_BLOCKING_JOB_NAMES)

src/Backups/BackupEntriesCollector.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -397,7 +397,7 @@ void BackupEntriesCollector::gatherDatabasesMetadata()
397397

398398
case ASTBackupQuery::ElementType::ALL:
399399
{
400-
for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases())
400+
for (const auto & [database_name, database] : DatabaseCatalog::instance().getDatabases(GetDatabasesOptions{.with_datalake_catalogs = true}))
401401
{
402402
if (!element.except_databases.contains(database_name))
403403
{

src/Core/Settings.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -6472,7 +6472,7 @@ Query Iceberg table using the snapshot that was current at a specific timestamp.
64726472
DECLARE(Int64, iceberg_snapshot_id, 0, R"(
64736473
Query Iceberg table using the specific snapshot id.
64746474
)", 0) \
6475-
DECLARE(Bool, show_data_lake_catalogs_in_system_tables, true, R"(
6475+
DECLARE(Bool, show_data_lake_catalogs_in_system_tables, false, R"(
64766476
Enables showing data lake catalogs in system tables.
64776477
)", 0) \
64786478
DECLARE(Bool, delta_lake_enable_expression_visitor_logging, false, R"(

src/Core/SettingsChangesHistory.cpp

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,11 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory()
3939
/// controls new feature and it's 'true' by default, use 'false' as previous_value).
4040
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
4141
/// Note: please check if the key already exists to prevent duplicate entries.
42+
addSettingsChanges(settings_changes_history, "25.8.13.10000",
43+
{
44+
{"show_data_lake_catalogs_in_system_tables", false, true, "Disable catalogs in system tables by default"},
45+
46+
});
4247
addSettingsChanges(settings_changes_history, "25.8",
4348
{
4449
{"output_format_json_quote_64bit_integers", true, false, "Disable quoting of the 64 bit integers in JSON by default"},

src/Databases/DataLake/DatabaseDataLake.cpp

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -532,14 +532,10 @@ DatabaseTablesIteratorPtr DatabaseDataLake::getTablesIterator(
532532
DatabaseTablesIteratorPtr DatabaseDataLake::getLightweightTablesIterator(
533533
ContextPtr context_,
534534
const FilterByNameFunction & filter_by_table_name,
535-
bool skip_not_loaded,
536-
bool skip_data_lake_catalog) const
535+
bool skip_not_loaded) const
537536
{
538537
Tables tables;
539538

540-
if (skip_data_lake_catalog)
541-
return std::make_unique<DatabaseTablesSnapshotIterator>(tables, getDatabaseName());
542-
543539
auto catalog = getCatalog();
544540
DB::Names iceberg_tables;
545541

src/Databases/DataLake/DatabaseDataLake.h

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@ class DatabaseDataLake final : public IDatabase, WithContext
3131
bool canContainDistributedTables() const override { return false; }
3232
bool canContainRocksDBTables() const override { return false; }
3333
bool shouldBeEmptyOnDetach() const override { return false; }
34+
bool isDatalakeCatalog() const override { return true; }
3435

3536
bool empty() const override;
3637

@@ -47,8 +48,7 @@ class DatabaseDataLake final : public IDatabase, WithContext
4748
DatabaseTablesIteratorPtr getLightweightTablesIterator(
4849
ContextPtr context,
4950
const FilterByNameFunction & filter_by_table_name,
50-
bool skip_not_loaded,
51-
bool skip_data_lake_catalog) const override;
51+
bool skip_not_loaded) const override;
5252

5353

5454
void shutdown() override {}

src/Databases/IDatabase.h

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -179,6 +179,8 @@ class IDatabase : public std::enable_shared_from_this<IDatabase>
179179

180180
virtual bool canContainRocksDBTables() const { return true; }
181181

182+
virtual bool isDatalakeCatalog() const { return false; }
183+
182184
/// Load a set of existing tables.
183185
/// You can call only once, right after the object is created.
184186
virtual void loadStoredObjects( /// NOLINT
@@ -267,7 +269,7 @@ class IDatabase : public std::enable_shared_from_this<IDatabase>
267269

268270
/// Same as above, but may return non-fully initialized StoragePtr objects which are not suitable for reading.
269271
/// Useful for queries like "SHOW TABLES"
270-
virtual DatabaseTablesIteratorPtr getLightweightTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}, bool skip_not_loaded = false, [[maybe_unused]] bool skip_data_lake_catalog = false) const /// NOLINT
272+
virtual DatabaseTablesIteratorPtr getLightweightTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}, bool skip_not_loaded = false) const /// NOLINT
271273
{
272274
return getTablesIterator(context, filter_by_table_name, skip_not_loaded);
273275
}

0 commit comments

Comments
 (0)