From 7aaf7e0e03d1d1e192a68277c4f6836ce92587e4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 10:44:28 +0200 Subject: [PATCH 01/54] Fix 02488_zero_copy_detached_parts_drop_table test --- .../0_stateless/02488_zero_copy_detached_parts_drop_table.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh index b01f16e1cad..60cec5caea3 100755 --- a/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh +++ b/tests/queries/0_stateless/02488_zero_copy_detached_parts_drop_table.sh @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is rm -f $path/count.txt $CLICKHOUSE_CLIENT -q "detach table rmt2 sync" -$CLICKHOUSE_CLIENT --send_logs_level='fatal' -q "attach table rmt2" +$CLICKHOUSE_CLIENT --allow_repeated_settings --send_logs_level='fatal' -q "attach table rmt2" $CLICKHOUSE_CLIENT -q "select reason, name from system.detached_parts where database='$CLICKHOUSE_DATABASE' and table='rmt2'" From 6556bd28dcb7912add1c192e318f4e9dd0f9cd31 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 15:23:36 +0200 Subject: [PATCH 02/54] Fix parallel write for Azure --- src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f52ab803012..cd154ec0f79 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -163,6 +163,8 @@ public: return client.get(); } + bool supportParallelWrite() const override { return true; } + private: using SharedAzureClientPtr = std::shared_ptr; void removeObjectImpl(const StoredObject & object, const SharedAzureClientPtr & client_ptr, bool if_exists); From 697b5556a3e16e8b0cbba43d5226a2003ef21f0a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 30 Apr 2024 22:10:36 +0200 Subject: [PATCH 03/54] Enable azure test --- tests/ci/functional_test_check.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index e898138fb3a..6418ee4e0ce 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -318,9 +318,6 @@ def main(): state, description, test_results, additional_logs = process_results( result_path, server_log_path ) - # FIXME (alesapin) - if "azure" in check_name: - state = "success" else: print( "This is validate bugfix or flaky check run, but no changes test to run - skip with success" From ccabc292ad6647855d5b99c1631b8ee77413b855 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 1 May 2024 08:48:24 +0200 Subject: [PATCH 04/54] Temporarily remove running azure tests only for master --- tests/ci/ci_config.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index bdfff12db0b..f7b32e9177c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1205,7 +1205,7 @@ CI_CONFIG = CIConfig( ), JobNames.STATELESS_TEST_AZURE_ASAN: TestConfig( Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **statless_test_common_params, release_only=True), # type: ignore + job_config=JobConfig(num_batches=4, **statless_test_common_params), # type: ignore ), JobNames.STATELESS_TEST_S3_TSAN: TestConfig( Build.PACKAGE_TSAN, @@ -1230,7 +1230,7 @@ CI_CONFIG = CIConfig( Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore ), JobNames.STRESS_TEST_AZURE_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params, release_only=True) # type: ignore + Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params) # type: ignore ), JobNames.UPGRADE_TEST_TSAN: TestConfig( Build.PACKAGE_TSAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore From 068ddc2d65f2ba4bce98aaafe134a94f843d5974 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 1 May 2024 11:03:52 +0200 Subject: [PATCH 05/54] Add allow_repeated_settings before appending send_logs_level in tests --- .../0_stateless/02494_query_cache_nested_query_bug.sh | 4 ++-- .../0_stateless/02535_max_parallel_replicas_custom_key.sh | 2 +- .../0_stateless/02922_deduplication_with_zero_copy.sh | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index 8712c7c84c6..59b107d06fe 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -19,7 +19,7 @@ SETTINGS="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --allow_repeated_settings --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 9850406eb3a..480517bf17f 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --allow_repeated_settings --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh index bb013dccb65..dad4e6747e0 100755 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh @@ -58,9 +58,9 @@ function filter_temporary_locks() function insert_duplicates() { - $CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --send_logs_level="error" & + $CLICKHOUSE_CLIENT -q "insert into r1 values(1);" --allow_repeated_settings --send_logs_level="error" & - $CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --send_logs_level="error" + $CLICKHOUSE_CLIENT -q "insert into r2 values(1);" --allow_repeated_settings --send_logs_level="error" wait @@ -137,8 +137,8 @@ function list_keeper_nodes() { list_keeper_nodes "${table_shared_id}" -$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --send_logs_level="error" & -$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r1;" --allow_repeated_settings --send_logs_level="error" & +$CLICKHOUSE_CLIENT -nm -q "drop table r2;" --allow_repeated_settings --send_logs_level="error" & wait list_keeper_nodes "${table_shared_id}" From 042c36313da07f77d3f2acae09d2ffb858fbfe37 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 16 May 2024 14:42:26 +0200 Subject: [PATCH 06/54] Updated to use required memory and prevent creating extra buffer --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 127 +++++++++++++++--- .../IO/WriteBufferFromAzureBlobStorage.h | 14 +- 2 files changed, 120 insertions(+), 21 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 2c90e3a9003..ebb3fc4a82d 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -23,7 +23,6 @@ struct WriteBufferFromAzureBlobStorage::PartData { Memory<> memory; size_t data_size = 0; - std::string block_id; }; BufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureObjectStorageSettings & settings) @@ -119,22 +118,32 @@ void WriteBufferFromAzureBlobStorage::preFinalize() // This function should not be run again is_prefinalized = true; + hidePartialData(); + + if (hidden_size > 0) + detachBuffer(); + + setFakeBufferWhenPreFinalized(); + /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (buffer_allocation_policy->getBufferNumber() == 1) + if (detached_part_data.size() == 1) { - size_t data_size = size_t(position() - memory.data()); - if (data_size <= max_single_part_upload_size) + if (detached_part_data.front().data_size <= max_single_part_upload_size) { + auto part_data = std::move(detached_part_data.front()); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(memory.data()), data_size); - execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, data_size); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); + execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); + + detached_part_data.pop_front(); + block_ids.clear(); return; } } - writePart(); + writeMultipartUpload(); } void WriteBufferFromAzureBlobStorage::finalizeImpl() @@ -144,9 +153,13 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() if (!is_prefinalized) preFinalize(); + chassert(offset() == 0); + chassert(hidden_size == 0); + + task_tracker->waitAll(); + if (!block_ids.empty()) { - task_tracker->waitAll(); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); @@ -156,13 +169,60 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() void WriteBufferFromAzureBlobStorage::nextImpl() { task_tracker->waitIfAny(); - writePart(); + + hidePartialData(); + + reallocateFirstBuffer(); + + if (available() > 0) + return; + + detachBuffer(); + + if (detached_part_data.size() > 1) + writeMultipartUpload(); + allocateBuffer(); } +void WriteBufferFromAzureBlobStorage::hidePartialData() +{ + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(offset(), ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + + chassert(memory.size() >= hidden_size + offset()); + + hidden_size += offset(); + chassert(memory.data() + hidden_size == working_buffer.begin() + offset()); + chassert(memory.data() + hidden_size == position()); + + WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); + chassert(offset() == 0); +} + +void WriteBufferFromAzureBlobStorage::reallocateFirstBuffer() +{ + chassert(offset() == 0); + + if (buffer_allocation_policy->getBufferNumber() > 1 || available() > 0) + return; + + const size_t max_first_buffer = buffer_allocation_policy->getBufferSize(); + if (memory.size() == max_first_buffer) + return; + + size_t size = std::min(memory.size() * 2, max_first_buffer); + memory.resize(size); + + WriteBuffer::set(memory.data() + hidden_size, memory.size() - hidden_size); + chassert(offset() == 0); +} + void WriteBufferFromAzureBlobStorage::allocateBuffer() { buffer_allocation_policy->nextBuffer(); + chassert(0 == hidden_size); + auto size = buffer_allocation_policy->getBufferSize(); if (buffer_allocation_policy->getBufferNumber() == 1) @@ -172,30 +232,57 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() WriteBuffer::set(memory.data(), memory.size()); } -void WriteBufferFromAzureBlobStorage::writePart() +void WriteBufferFromAzureBlobStorage::detachBuffer() { - auto data_size = size_t(position() - memory.data()); + size_t data_size = size_t(position() - memory.data()); if (data_size == 0) return; - const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); - std::shared_ptr part_data = std::make_shared(std::move(memory), data_size, block_id); - WriteBuffer::set(nullptr, 0); + chassert(data_size == hidden_size); - auto upload_worker = [this, part_data] () + auto buf = std::move(memory); + + WriteBuffer::set(nullptr, 0); + total_size += hidden_size; + hidden_size = 0; + + detached_part_data.push_back({std::move(buf), data_size}); + WriteBuffer::set(nullptr, 0); +} + +void WriteBufferFromAzureBlobStorage::writePart(WriteBufferFromAzureBlobStorage::PartData && part_data) +{ + const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); + auto worker_data = std::make_shared>(block_id, std::move(part_data)); + + auto upload_worker = [this, worker_data] () { + auto & data_size = std::get<1>(*worker_data).data_size; + auto & data_block_id = std::get<0>(*worker_data); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data->memory.data()), part_data->data_size); - execWithRetry([&](){ block_blob_client.StageBlock(part_data->block_id, memory_stream); }, max_unexpected_write_error_retries, part_data->data_size); - - if (write_settings.remote_throttler) - write_settings.remote_throttler->add(part_data->data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(std::get<1>(*worker_data).memory.data()), data_size); + execWithRetry([&](){ block_blob_client.StageBlock(data_block_id, memory_stream); }, max_unexpected_write_error_retries, data_size); }; task_tracker->add(std::move(upload_worker)); } +void WriteBufferFromAzureBlobStorage::setFakeBufferWhenPreFinalized() +{ + WriteBuffer::set(fake_buffer_when_prefinalized, sizeof(fake_buffer_when_prefinalized)); +} + +void WriteBufferFromAzureBlobStorage::writeMultipartUpload() +{ + while (!detached_part_data.empty()) + { + writePart(std::move(detached_part_data.front())); + detached_part_data.pop_front(); + } +} + } #endif + diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 96ba6acefff..25d515df8c8 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -48,8 +48,13 @@ public: private: struct PartData; - void writePart(); + void writeMultipartUpload(); + void writePart(PartData && part_data); + void detachBuffer(); + void reallocateFirstBuffer(); void allocateBuffer(); + void hidePartialData(); + void setFakeBufferWhenPreFinalized(); void finalizeImpl() override; void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); @@ -77,9 +82,16 @@ private: MemoryBufferPtr allocateBuffer() const; + char fake_buffer_when_prefinalized[1] = {}; + bool first_buffer=true; + size_t total_size = 0; + size_t hidden_size = 0; + std::unique_ptr task_tracker; + + std::deque detached_part_data; }; } From 6510f804478303fb5d420d43459201018bea6dc1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 23 May 2024 16:59:07 +0200 Subject: [PATCH 07/54] Updated test to fix unrecognized option error --- tests/queries/0_stateless/02941_variant_type_1.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_2.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_3.sh | 6 +++--- tests/queries/0_stateless/02941_variant_type_4.sh | 6 +++--- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02941_variant_type_1.sh b/tests/queries/0_stateless/02941_variant_type_1.sh index 22ca909a26e..723de45eaad 100755 --- a/tests/queries/0_stateless/02941_variant_type_1.sh +++ b/tests/queries/0_stateless/02941_variant_type_1.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test1_insert() { @@ -115,11 +115,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh index 91ba0285bd8..f43cd2bb0d6 100755 --- a/tests/queries/0_stateless/02941_variant_type_2.sh +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test4_insert() { @@ -61,11 +61,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_3.sh b/tests/queries/0_stateless/02941_variant_type_3.sh index 8a039a02d6d..f4b2b304f56 100755 --- a/tests/queries/0_stateless/02941_variant_type_3.sh +++ b/tests/queries/0_stateless/02941_variant_type_3.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test5_insert() { @@ -63,11 +63,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index b003bcdcef1..f9a16847864 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1 --index_granularity_bytes=10485760 --index_granularity=8192 " +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test6_insert() { @@ -58,11 +58,11 @@ run 0 $CH_CLIENT -q "drop table test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +$CH_CLIENT -q "create table test (id UInt64, v Variant(String, UInt64, LowCardinality(String), Tuple(a UInt32, b UInt32), Array(UInt64))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, index_granularity_bytes=10485760, index_granularity=8192;" run 1 $CH_CLIENT -q "drop table test;" From 73f42b0858204f1682269453b0565380afc7a9f4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 May 2024 20:59:10 +0200 Subject: [PATCH 08/54] add clusters with replicas from all replica groups --- src/Databases/DatabaseReplicated.cpp | 67 +++++++++++++++---- src/Databases/DatabaseReplicated.h | 6 +- src/Databases/DatabaseReplicatedWorker.cpp | 2 + src/Interpreters/DDLTask.cpp | 17 ++++- src/Storages/System/StorageSystemClusters.cpp | 4 ++ 5 files changed, 78 insertions(+), 18 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index cc946fc22c4..c9e14790175 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -173,13 +173,40 @@ ClusterPtr DatabaseReplicated::tryGetCluster() const return cluster; } -void DatabaseReplicated::setCluster(ClusterPtr && new_cluster) +ClusterPtr DatabaseReplicated::tryGetAllGroupsCluster() const { std::lock_guard lock{mutex}; - cluster = std::move(new_cluster); + if (replica_group_name.empty()) + return nullptr; + + if (cluster_all_groups) + return cluster_all_groups; + + /// Database is probably not created or not initialized yet, it's ok to return nullptr + if (is_readonly) + return cluster_all_groups; + + try + { + cluster_all_groups = getClusterImpl(/*all_groups*/ true); + } + catch (...) + { + tryLogCurrentException(log); + } + return cluster_all_groups; } -ClusterPtr DatabaseReplicated::getClusterImpl() const +void DatabaseReplicated::setCluster(ClusterPtr && new_cluster, bool all_groups) +{ + std::lock_guard lock{mutex}; + if (all_groups) + cluster_all_groups = std::move(new_cluster); + else + cluster = std::move(new_cluster); +} + +ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const { Strings unfiltered_hosts; Strings hosts; @@ -199,17 +226,24 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const "It's possible if the first replica is not fully created yet " "or if the last replica was just dropped or due to logical error", zookeeper_path); - hosts.clear(); - std::vector paths; - for (const auto & host : unfiltered_hosts) - paths.push_back(zookeeper_path + "/replicas/" + host + "/replica_group"); - - auto replica_groups = zookeeper->tryGet(paths); - - for (size_t i = 0; i < paths.size(); ++i) + if (all_groups) { - if (replica_groups[i].data == replica_group_name) - hosts.push_back(unfiltered_hosts[i]); + hosts = unfiltered_hosts; + } + else + { + hosts.clear(); + std::vector paths; + for (const auto & host : unfiltered_hosts) + paths.push_back(zookeeper_path + "/replicas/" + host + "/replica_group"); + + auto replica_groups = zookeeper->tryGet(paths); + + for (size_t i = 0; i < paths.size(); ++i) + { + if (replica_groups[i].data == replica_group_name) + hosts.push_back(unfiltered_hosts[i]); + } } Int32 cversion = stat.cversion; @@ -274,6 +308,11 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const bool treat_local_as_remote = false; bool treat_local_port_as_remote = getContext()->getApplicationType() == Context::ApplicationType::LOCAL; + + String cluster_name = TSA_SUPPRESS_WARNING_FOR_READ(database_name); /// FIXME + if (all_groups) + cluster_name = "all_groups." + cluster_name; + ClusterConnectionParameters params{ cluster_auth_info.cluster_username, cluster_auth_info.cluster_password, @@ -282,7 +321,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const treat_local_port_as_remote, cluster_auth_info.cluster_secure_connection, Priority{1}, - TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME + cluster_name, cluster_auth_info.cluster_secret}; return std::make_shared(getContext()->getSettingsRef(), shards, params); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 55bcf963d37..f902b45ca86 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -65,6 +65,7 @@ public: /// Returns cluster consisting of database replicas ClusterPtr tryGetCluster() const; + ClusterPtr tryGetAllGroupsCluster() const; void drop(ContextPtr /*context*/) override; @@ -113,8 +114,8 @@ private: ASTPtr parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query); String readMetadataFile(const String & table_name) const; - ClusterPtr getClusterImpl() const; - void setCluster(ClusterPtr && new_cluster); + ClusterPtr getClusterImpl(bool all_groups = false) const; + void setCluster(ClusterPtr && new_cluster, bool all_groups = false); void createEmptyLogEntry(const ZooKeeperPtr & current_zookeeper); @@ -155,6 +156,7 @@ private: UInt64 tables_metadata_digest TSA_GUARDED_BY(metadata_mutex); mutable ClusterPtr cluster; + mutable ClusterPtr cluster_all_groups; LoadTaskPtr startup_replicated_database_task TSA_GUARDED_BY(mutex); }; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 6e19a77c501..31d6f7876a8 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -421,6 +421,8 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na { /// Some replica is added or removed, let's update cached cluster database->setCluster(database->getClusterImpl()); + if (!database->replica_group_name.empty()) + database->setCluster(database->getClusterImpl(/*all_groups*/ true), /*all_groups*/ true); out_reason = fmt::format("Entry {} is a dummy task", entry_name); return {}; } diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index a37b4db029a..06ec9489fc1 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -568,8 +568,21 @@ void ZooKeeperMetadataTransaction::commit() ClusterPtr tryGetReplicatedDatabaseCluster(const String & cluster_name) { - if (const auto * replicated_db = dynamic_cast(DatabaseCatalog::instance().tryGetDatabase(cluster_name).get())) - return replicated_db->tryGetCluster(); + String name = cluster_name; + bool all_groups = false; + if (name.starts_with("all_groups.")) + { + name = name.substr(strlen("all_groups.")); + all_groups = true; + } + + if (const auto * replicated_db = dynamic_cast(DatabaseCatalog::instance().tryGetDatabase(name).get())) + { + if (all_groups) + return replicated_db->tryGetAllGroupsCluster(); + else + return replicated_db->tryGetCluster(); + } return {}; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index cb8d5caa50c..520bd7e7f92 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -54,6 +54,10 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co if (auto database_cluster = replicated->tryGetCluster()) writeCluster(res_columns, {name_and_database.first, database_cluster}, replicated->tryGetAreReplicasActive(database_cluster)); + + if (auto database_cluster = replicated->tryGetAllGroupsCluster()) + writeCluster(res_columns, {"all_groups." + name_and_database.first, database_cluster}, + replicated->tryGetAreReplicasActive(database_cluster)); } } } From aa93ace6d8d92cca4dedcf5a2bebf7c8b2b966d6 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 24 May 2024 10:42:59 +0200 Subject: [PATCH 09/54] Fixed more tests with unrecognized option error --- tests/queries/0_stateless/02941_variant_type_4.sh | 2 +- .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- .../0_stateless/03039_dynamic_all_merge_algorithms_2.sh | 6 +++--- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index b90f9090b07..f9a16847864 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1"g +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspicious_variant_types=1" function test6_insert() { diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 7c1ac41cfdc..0e4c2a0cc5e 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760 --index_granularity 8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index 927ceac72b5..e47b11614af 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760 --index_granularity 8192" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh index 02362012960..0be0e973d01 100755 --- a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh +++ b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --index_granularity_bytes 10485760 --index_granularity 8128 --merge_max_block_size 8128" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8128" function test() @@ -44,7 +44,7 @@ echo "MergeTree wide + horizontal merge" test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1" echo "MergeTree compact + vertical merge" -test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1" +test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" echo "MergeTree wide + vertical merge" -test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1" +test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" From d2739d495157026fa3df6e7c140bb2800fad7cea Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 24 May 2024 13:37:57 +0200 Subject: [PATCH 10/54] Fixed merge_max_block_size setting --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 1 + .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- .../0_stateless/03039_dynamic_all_merge_algorithms_2.sh | 6 +++--- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index caa600298ce..b100f96befa 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -24,6 +24,7 @@ function check_refcnt_for_table() local log_file log_file=$(mktemp "$CUR_DIR/clickhouse-tests.XXXXXX.log") local args=( + --allow_repeated_settings --format Null --max_threads 1 --max_block_size 1 diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 0e4c2a0cc5e..0c9060d261a 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index e47b11614af..2384350fa85 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8192 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh index 0be0e973d01..68cc72faf59 100755 --- a/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh +++ b/tests/queries/0_stateless/03039_dynamic_all_merge_algorithms_2.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size 8128" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() @@ -44,7 +44,7 @@ echo "MergeTree wide + horizontal merge" test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1" echo "MergeTree compact + vertical merge" -test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" +test "min_rows_for_wide_part=100000000000, min_bytes_for_wide_part=1000000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128, merge_max_block_size=8128;" echo "MergeTree wide + vertical merge" -test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128;" +test "min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8128, merge_max_block_size=8128;" From 6725168b983d197d4e00234785e02373327fd30c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 May 2024 21:34:33 +0200 Subject: [PATCH 11/54] better code, add warning --- src/Databases/DatabaseReplicated.cpp | 9 ++++++++- src/Databases/DatabaseReplicated.h | 2 ++ src/Interpreters/DDLTask.cpp | 4 ++-- src/Storages/System/StorageSystemClusters.cpp | 2 +- 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index c9e14790175..d2a3a5d421c 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -122,6 +122,13 @@ DatabaseReplicated::DatabaseReplicated( fillClusterAuthInfo(db_settings.collection_name.value, context_->getConfigRef()); replica_group_name = context_->getConfigRef().getString("replica_group_name", ""); + + if (!replica_group_name.empty() && database_name.starts_with(DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX)) + { + context_->addWarningMessage(fmt::format("There's a Replicated database with a name starting from '{}', " + "and replica_group_name is configured. It may cause collisions in cluster names.", + ALL_GROUPS_CLUSTER_PREFIX)); + } } String DatabaseReplicated::getFullReplicaName(const String & shard, const String & replica) @@ -311,7 +318,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl(bool all_groups) const String cluster_name = TSA_SUPPRESS_WARNING_FOR_READ(database_name); /// FIXME if (all_groups) - cluster_name = "all_groups." + cluster_name; + cluster_name = ALL_GROUPS_CLUSTER_PREFIX + cluster_name; ClusterConnectionParameters params{ cluster_auth_info.cluster_username, diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index f902b45ca86..761d6b4b503 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -20,6 +20,8 @@ using ClusterPtr = std::shared_ptr; class DatabaseReplicated : public DatabaseAtomic { public: + static constexpr auto ALL_GROUPS_CLUSTER_PREFIX = "all_groups."; + DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid, const String & zookeeper_path_, const String & shard_name_, const String & replica_name_, DatabaseReplicatedSettings db_settings_, diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 06ec9489fc1..6c346836ed8 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -570,9 +570,9 @@ ClusterPtr tryGetReplicatedDatabaseCluster(const String & cluster_name) { String name = cluster_name; bool all_groups = false; - if (name.starts_with("all_groups.")) + if (name.starts_with(DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX)) { - name = name.substr(strlen("all_groups.")); + name = name.substr(strlen(DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX)); all_groups = true; } diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 520bd7e7f92..160c8d6270e 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -56,7 +56,7 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co replicated->tryGetAreReplicasActive(database_cluster)); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {"all_groups." + name_and_database.first, database_cluster}, + writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, replicated->tryGetAreReplicasActive(database_cluster)); } } From 8484decf4891c7c8939688e3261ea3b5e4c53584 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 27 May 2024 10:32:17 +0200 Subject: [PATCH 12/54] Fix test fails --- .../0_stateless/03037_dynamic_merges_1_horizontal.sh | 6 +++--- .../queries/0_stateless/03037_dynamic_merges_1_vertical.sh | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh index 0c9060d261a..887b2ed94d7 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_horizontal.sh @@ -8,7 +8,7 @@ CLICKHOUSE_LOG_COMMENT= . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() { @@ -41,12 +41,12 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" diff --git a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh index 2384350fa85..371ae87c2ef 100755 --- a/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh +++ b/tests/queries/0_stateless/03037_dynamic_merges_1_vertical.sh @@ -9,7 +9,7 @@ CLICKHOUSE_LOG_COMMENT= -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --merge_max_block_size_bytes=10485760" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1" function test() { echo "test" @@ -41,11 +41,11 @@ function test() $CH_CLIENT -q "drop table if exists test;" echo "MergeTree compact" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" echo "MergeTree wide" -$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192;" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic(max_types=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;" test $CH_CLIENT -q "drop table test;" From ef3b802b4e543cb879fe7c45eb1f57423c3e471e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 May 2024 18:42:10 +0200 Subject: [PATCH 13/54] add test --- .../configs/config2.xml | 10 ++++++++ .../test_replicated_database/test.py | 23 ++++++++++++++++++- 2 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_replicated_database/configs/config2.xml diff --git a/tests/integration/test_replicated_database/configs/config2.xml b/tests/integration/test_replicated_database/configs/config2.xml new file mode 100644 index 00000000000..727461697ca --- /dev/null +++ b/tests/integration/test_replicated_database/configs/config2.xml @@ -0,0 +1,10 @@ + + 10 + 1 + + 10 + + 50 + 42 + group + diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index fd1bfc75227..ef86da5af30 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -46,7 +46,7 @@ snapshotting_node = cluster.add_instance( ) snapshot_recovering_node = cluster.add_instance( "snapshot_recovering_node", - main_configs=["configs/config.xml"], + main_configs=["configs/config2.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, ) @@ -1522,3 +1522,24 @@ def test_auto_recovery(started_cluster): assert "42\n" == bad_settings_node.query("SELECT * FROM auto_recovery.t2") assert "137\n" == bad_settings_node.query("SELECT * FROM auto_recovery.t1") + + +def test_all_groups_cluster(started_cluster): + dummy_node.query("DROP DATABASE IF EXISTS db_cluster") + bad_settings_node.query("DROP DATABASE IF EXISTS db_cluster") + dummy_node.query( + "CREATE DATABASE db_cluster ENGINE = Replicated('/clickhouse/databases/all_groups_cluster', 'shard1', 'replica1');" + ) + bad_settings_node.query( + "CREATE DATABASE db_cluster ENGINE = Replicated('/clickhouse/databases/all_groups_cluster', 'shard1', 'replica2');" + ) + + assert "bad_settings_node\ndummy_node\n" == dummy_node.query( + "select host_name from system.clusters where name='db_cluster' order by host_name" + ) + assert "bad_settings_node\n" == bad_settings_node.query( + "select host_name from system.clusters where name='db_cluster' order by host_name" + ) + assert "bad_settings_node\ndummy_node\n" == bad_settings_node.query( + "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" + ) From 35d3cddf993822b31a9cdf21418295f9289be961 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 30 May 2024 21:36:12 +0200 Subject: [PATCH 14/54] Added condition for single block commit --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index ebb3fc4a82d..8333bf45e67 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -127,7 +127,7 @@ void WriteBufferFromAzureBlobStorage::preFinalize() /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (detached_part_data.size() == 1) + if (detached_part_data.size() == 1 && block_ids.empty()) { if (detached_part_data.front().data_size <= max_single_part_upload_size) { @@ -168,6 +168,11 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() void WriteBufferFromAzureBlobStorage::nextImpl() { + if (is_prefinalized) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot write to prefinalized buffer for Azure Blob Storage, the file could have been created"); + task_tracker->waitIfAny(); hidePartialData(); From 216c67c9f8cb8dcb79fd061f92bc3e9576be1fd9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 31 May 2024 10:05:15 +0200 Subject: [PATCH 15/54] Fixed style check --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 8333bf45e67..16bd70cf2e5 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -19,6 +19,11 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + struct WriteBufferFromAzureBlobStorage::PartData { Memory<> memory; From d2184fd2a2d76de2de2252b9152c08f7ae269574 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 3 Jun 2024 20:51:09 +0200 Subject: [PATCH 16/54] Update test.py --- tests/integration/test_replicated_database/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index ef86da5af30..a5859960cd9 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -46,7 +46,7 @@ snapshotting_node = cluster.add_instance( ) snapshot_recovering_node = cluster.add_instance( "snapshot_recovering_node", - main_configs=["configs/config2.xml"], + main_configs=["configs/config.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, ) @@ -61,7 +61,7 @@ all_nodes = [ bad_settings_node = cluster.add_instance( "bad_settings_node", - main_configs=["configs/config.xml"], + main_configs=["configs/config2.xml"], user_configs=["configs/inconsistent_settings.xml"], with_zookeeper=True, macros={"shard": 1, "replica": 4}, From 3d7beae8fd4140f90917a592f3aae1bfecb90c5a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 4 Jun 2024 19:29:26 +0200 Subject: [PATCH 17/54] Simplify 03023_zeros_generate_random_with_limit_progress_bar --- ...rate_random_with_limit_progress_bar.expect | 49 ------------------- ...e_random_with_limit_progress_bar.reference | 3 ++ ...generate_random_with_limit_progress_bar.sh | 16 ++++++ 3 files changed, 19 insertions(+), 49 deletions(-) delete mode 100755 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect create mode 100755 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect deleted file mode 100755 index de15a199132..00000000000 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/expect -f - -set basedir [file dirname $argv0] -set basename [file tail $argv0] -if {[info exists env(CLICKHOUSE_TMP)]} { - set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP) -} else { - set CLICKHOUSE_TMP "." -} -exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 - -log_user 0 -set timeout 60 -match_max 100000 -set stty_init "rows 25 cols 120" - -expect_after { - -i $any_spawn_id eof { exp_continue } - -i $any_spawn_id timeout { exit 1 } -} - -spawn clickhouse-local -expect ":) " - -# Trivial SELECT with LIMIT from system.zeros shows progress bar. -send "SELECT * FROM system.zeros LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -send "SELECT * FROM system.zeros_mt LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -# As well as from generateRandom -send "SELECT * FROM generateRandom() LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" -expect "Progress: " -expect "█" -send "\3" -expect "Query was cancelled." -expect ":) " - -send "exit\r" -expect eof diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference index e69de29bb2d..6ca5ae94f9a 100644 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference @@ -0,0 +1,3 @@ +Matched +Matched +Matched diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh new file mode 100755 index 00000000000..4bb8fc8880d --- /dev/null +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function run_with_progress_and_match_total_rows() +{ + echo "$1" | \ + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1 | \ + grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found" +} + +run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros LIMIT 100' +run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros_mt LIMIT 100' +run_with_progress_and_match_total_rows 'SELECT * FROM generateRandom() LIMIT 100' From 553fcb5e0618858cfa7659522208be49320b0e48 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 4 Jun 2024 20:05:58 +0200 Subject: [PATCH 18/54] Update test.py --- tests/integration/test_replicated_database/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index a5859960cd9..ea569939c1c 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1534,7 +1534,7 @@ def test_all_groups_cluster(started_cluster): "CREATE DATABASE db_cluster ENGINE = Replicated('/clickhouse/databases/all_groups_cluster', 'shard1', 'replica2');" ) - assert "bad_settings_node\ndummy_node\n" == dummy_node.query( + assert "dummy_node\n" == dummy_node.query( "select host_name from system.clusters where name='db_cluster' order by host_name" ) assert "bad_settings_node\n" == bad_settings_node.query( From d99f8b4acc3991cbcb8c269fc27773146d47c6cf Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 5 Jun 2024 20:35:22 +0330 Subject: [PATCH 19/54] Added Named collections for accessing Kafka --- docs/en/operations/named-collections.md | 56 +++++++++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index c9d94dd95ee..5ba9029152e 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -443,3 +443,59 @@ SELECT dictGet('dict', 'b', 1); │ a │ └─────────────────────────┘ ``` + +## Named collections for accessing Kafka + +The description of parameters see [Kafka](../engines/table-engines/integrations/kafka.md). + +### DDL example + +```sql +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka SETTINGS kafka_broker_list = 'localhost:9092', + kafka_topic_list = 'kafka_topic', + kafka_group_name = 'consumer_group', + kafka_format = 'JSONEachRow', + kafka_max_block_size = 1048576; +``` +### XML example + +```xml + + + + localhost:9092 + kafka_topic + consumer_group + JSONEachRow + 1048576 + + + +``` + +### kafka Table, named collection examples + +Both of the following examples use the same named collection `my_kafka_cluster`: + + +```sql +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka(my_kafka_cluster) + + +CREATE TABLE queue( + timestamp UInt64, + level String, + message String +)ENGINE = Kafka(my_kafka_cluster) +SETTINGS kafka_num_consumers = 4, + kafka_thread_per_consumer = 1; + +``` From 46f14046e4e410f67ef7b436562a03c5bb605a98 Mon Sep 17 00:00:00 2001 From: xogoodnow Date: Wed, 5 Jun 2024 21:34:53 +0330 Subject: [PATCH 20/54] Added example on how to create named collection for kafka using ddl --- docs/en/operations/named-collections.md | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 5ba9029152e..9b9b682481d 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -451,15 +451,13 @@ The description of parameters see [Kafka](../engines/table-engines/integrations/ ### DDL example ```sql -CREATE TABLE queue( - timestamp UInt64, - level String, - message String -)ENGINE = Kafka SETTINGS kafka_broker_list = 'localhost:9092', - kafka_topic_list = 'kafka_topic', - kafka_group_name = 'consumer_group', - kafka_format = 'JSONEachRow', - kafka_max_block_size = 1048576; +CREATE NAMED COLLECTION my_kafka_cluster AS +kafka_broker_list = 'localhost:9092', +kafka_topic_list = 'kafka_topic', +kafka_group_name = 'consumer_group', +kafka_format = 'JSONEachRow', +kafka_max_block_size = '1048576'; + ``` ### XML example From 927720a02033239cb71568008c0a90f5e5482fc0 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:13 +0330 Subject: [PATCH 21/54] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 9b9b682481d..12d68b266c6 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -488,11 +488,13 @@ CREATE TABLE queue( )ENGINE = Kafka(my_kafka_cluster) -CREATE TABLE queue( +CREATE TABLE queue +( timestamp UInt64, level String, message String -)ENGINE = Kafka(my_kafka_cluster) +) +ENGINE = Kafka(my_kafka_cluster) SETTINGS kafka_num_consumers = 4, kafka_thread_per_consumer = 1; From 0316fa701fcaf2a81360fa0c7f16e4e83942d22d Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:23 +0330 Subject: [PATCH 22/54] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 12d68b266c6..4a3d303c114 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -475,7 +475,7 @@ kafka_max_block_size = '1048576'; ``` -### kafka Table, named collection examples +### Example of using named collections with a Kafka table Both of the following examples use the same named collection `my_kafka_cluster`: From 082efe2668bac79c13642ef4f9ea103e34fc0b62 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 21:40:28 +0330 Subject: [PATCH 23/54] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 4a3d303c114..a563ed626fe 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -481,11 +481,13 @@ Both of the following examples use the same named collection `my_kafka_cluster`: ```sql -CREATE TABLE queue( +CREATE TABLE queue +( timestamp UInt64, level String, message String -)ENGINE = Kafka(my_kafka_cluster) +) +ENGINE = Kafka(my_kafka_cluster) CREATE TABLE queue From 8aea0c3fecf852cbed7c2dd55f98777fcb71e97d Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:19 +0330 Subject: [PATCH 24/54] Omitted extra spaces Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index a563ed626fe..0438fcae3b2 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -465,11 +465,11 @@ kafka_max_block_size = '1048576'; - localhost:9092 - kafka_topic - consumer_group - JSONEachRow - 1048576 + localhost:9092 + kafka_topic + consumer_group + JSONEachRow + 1048576 From 844a94383b7473d2174075257c3a99fa4a54c978 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:36 +0330 Subject: [PATCH 25/54] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 0438fcae3b2..734b8dbe24a 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -489,7 +489,6 @@ CREATE TABLE queue ) ENGINE = Kafka(my_kafka_cluster) - CREATE TABLE queue ( timestamp UInt64, From 007a8ca32edba3b83f32ddffd95b4c2a13f78069 Mon Sep 17 00:00:00 2001 From: Ali <115415312+xogoodnow@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:05:45 +0330 Subject: [PATCH 26/54] Update docs/en/operations/named-collections.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/en/operations/named-collections.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 734b8dbe24a..1d261d935af 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -498,5 +498,4 @@ CREATE TABLE queue ENGINE = Kafka(my_kafka_cluster) SETTINGS kafka_num_consumers = 4, kafka_thread_per_consumer = 1; - ``` From 9ce824d41c2323f6556dd6f4015d9ec99272083c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 6 Jun 2024 11:04:38 +0200 Subject: [PATCH 27/54] Addressed review comments --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 27 +++++++++---------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 16bd70cf2e5..a2d21cf49c2 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -132,23 +132,21 @@ void WriteBufferFromAzureBlobStorage::preFinalize() /// If there is only one block and size is less than or equal to max_single_part_upload_size /// then we use single part upload instead of multi part upload - if (detached_part_data.size() == 1 && block_ids.empty()) + if (block_ids.empty() && detached_part_data.size() == 1 && detached_part_data.front().data_size <= max_single_part_upload_size) { - if (detached_part_data.front().data_size <= max_single_part_upload_size) - { - auto part_data = std::move(detached_part_data.front()); - auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); - execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); - LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); + auto part_data = std::move(detached_part_data.front()); + auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data.memory.data()), part_data.data_size); + execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, part_data.data_size); + LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); - detached_part_data.pop_front(); - block_ids.clear(); - return; - } + detached_part_data.pop_front(); + return; + } + else + { + writeMultipartUpload(); } - - writeMultipartUpload(); } void WriteBufferFromAzureBlobStorage::finalizeImpl() @@ -295,4 +293,3 @@ void WriteBufferFromAzureBlobStorage::writeMultipartUpload() } #endif - From 1d3cf170534b1ae8382397d8d2672868e237f33f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:11:08 +0200 Subject: [PATCH 28/54] Fix global trace collector --- programs/server/Server.cpp | 99 +++++++++++++++-------------- src/Common/QueryProfiler.cpp | 6 +- src/Interpreters/Context.cpp | 17 ++++- src/Interpreters/Context.h | 2 + src/Interpreters/TraceCollector.cpp | 30 +++++++-- src/Interpreters/TraceCollector.h | 11 +++- 6 files changed, 105 insertions(+), 60 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..f47cd53f9e9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -773,7 +773,51 @@ try LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); #endif - bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); + bool has_trace_collector = false; + /// Disable it if we collect test coverage information, because it will work extremely slow. +#if !WITH_COVERAGE + /// Profilers cannot work reliably with any other libunwind or without PHDR cache. + if (hasPHDRCache() && config().has("trace_log")) + { + has_trace_collector = true; + + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + { + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + } + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + { + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + } + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + { + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + } + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + { + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + } +#endif + + /// Describe multiple reasons when query profiler cannot work. + +#if WITH_COVERAGE + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); +#endif + +#if defined(SANITIZER) + LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" + " when two different stack unwinding methods will interfere with each other."); +#endif + + if (!hasPHDRCache()) + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" + " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will @@ -782,8 +826,12 @@ try server_settings.max_thread_pool_size, server_settings.max_thread_pool_free_size, server_settings.thread_pool_queue_size, - will_have_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, - will_have_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + has_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, + has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + + if (has_trace_collector) + global_context->createTraceCollector(); + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; @@ -1950,52 +1998,9 @@ try LOG_DEBUG(log, "Loaded metadata."); - /// Init trace collector only after trace_log system table was created - /// Disable it if we collect test coverage information, because it will work extremely slow. -#if !WITH_COVERAGE - /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache()) - { + if (has_trace_collector) global_context->initializeTraceCollector(); - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } -#endif - - /// Describe multiple reasons when query profiler cannot work. - -#if WITH_COVERAGE - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); -#endif - -#if defined(SANITIZER) - LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" - " when two different stack unwinding methods will interfere with each other."); -#endif - - if (!hasPHDRCache()) - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" - " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); - #if defined(OS_LINUX) auto tasks_stats_provider = TasksStatsCounters::findBestAvailableProvider(); if (tasks_stats_provider == TasksStatsCounters::MetricsProvider::None) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index c3affbdd968..746010b5462 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -228,9 +228,9 @@ void Timer::cleanup() #endif template -QueryProfilerBase::QueryProfilerBase([[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) - : log(getLogger("QueryProfiler")) - , pause_signal(pause_signal_) +QueryProfilerBase::QueryProfilerBase( + [[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) + : log(getLogger("QueryProfiler")), pause_signal(pause_signal_) { #if defined(SANITIZER) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 06b3adb328d..b1fb6a68618 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -740,12 +740,18 @@ struct ContextSharedPart : boost::noncopyable void initializeTraceCollector(std::shared_ptr trace_log) { - if (!trace_log) - return; + if (!trace_collector.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector needs to be first created before initialization"); + + trace_collector->initialize(trace_log); + } + + void createTraceCollector() + { if (hasTraceCollector()) return; - trace_collector.emplace(std::move(trace_log)); + trace_collector.emplace(); } void addWarningMessage(const String & message) TSA_REQUIRES(mutex) @@ -3891,6 +3897,11 @@ void Context::initializeSystemLogs() }); } +void Context::createTraceCollector() +{ + shared->createTraceCollector(); +} + void Context::initializeTraceCollector() { shared->initializeTraceCollector(getTraceLog()); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87a7baa0469..68f37377926 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1077,6 +1077,8 @@ public: void initializeSystemLogs(); /// Call after initialization before using trace collector. + void createTraceCollector(); + void initializeTraceCollector(); /// Call after unexpected crash happen. diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 8e9c397b7a1..77f70d754c8 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -1,5 +1,4 @@ -#include "TraceCollector.h" - +#include #include #include #include @@ -14,8 +13,12 @@ namespace DB { -TraceCollector::TraceCollector(std::shared_ptr trace_log_) - : trace_log(std::move(trace_log_)) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +TraceCollector::TraceCollector() { TraceSender::pipe.open(); @@ -28,6 +31,23 @@ TraceCollector::TraceCollector(std::shared_ptr trace_log_) thread = ThreadFromGlobalPool(&TraceCollector::run, this); } +void TraceCollector::initialize(std::shared_ptr trace_log_) +{ + if (is_trace_log_initialized) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector is already initialized"); + + trace_log_ptr = trace_log_; + is_trace_log_initialized.store(true, std::memory_order_release); +} + +std::shared_ptr TraceCollector::getTraceLog() +{ + if (!is_trace_log_initialized.load(std::memory_order_acquire)) + return nullptr; + + return trace_log_ptr; +} + void TraceCollector::tryClosePipe() { try @@ -120,7 +140,7 @@ void TraceCollector::run() ProfileEvents::Count increment; readPODBinary(increment, in); - if (trace_log) + if (auto trace_log = getTraceLog()) { // time and time_in_microseconds are both being constructed from the same timespec so that the // times will be equal up to the precision of a second. diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index 382e7511ac6..db1ce286b0e 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -1,4 +1,5 @@ #pragma once +#include #include class StackTrace; @@ -16,11 +17,17 @@ class TraceLog; class TraceCollector { public: - explicit TraceCollector(std::shared_ptr trace_log_); + explicit TraceCollector(); ~TraceCollector(); + void initialize(std::shared_ptr trace_log_); + private: - std::shared_ptr trace_log; + std::shared_ptr getTraceLog(); + + std::atomic is_trace_log_initialized = false; + std::shared_ptr trace_log_ptr; + ThreadFromGlobalPool thread; void tryClosePipe(); From fcee260b259b7750f18389e26debc90de22017a5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:22:48 +0200 Subject: [PATCH 29/54] Update src/Interpreters/TraceCollector.h Co-authored-by: alesapin --- src/Interpreters/TraceCollector.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index db1ce286b0e..c2894394dd0 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -17,7 +17,7 @@ class TraceLog; class TraceCollector { public: - explicit TraceCollector(); + TraceCollector(); ~TraceCollector(); void initialize(std::shared_ptr trace_log_); From bfb1c4c79396bb234b63a0a963fc643af292956d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 11:29:42 +0200 Subject: [PATCH 30/54] better --- programs/server/Server.cpp | 41 +++++++++++++++----------------------- 1 file changed, 16 insertions(+), 25 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f47cd53f9e9..84ff0e94cef 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -777,31 +777,7 @@ try /// Disable it if we collect test coverage information, because it will work extremely slow. #if !WITH_COVERAGE /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache() && config().has("trace_log")) - { - has_trace_collector = true; - - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } + has_trace_collector = hasPHDRCache() && config().has("trace_log"); #endif /// Describe multiple reasons when query profiler cannot work. @@ -830,8 +806,23 @@ try has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); if (has_trace_collector) + { global_context->createTraceCollector(); + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; From d0a2abe17be7381c261a6d631037bd3a57d4fdb1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 4 Jun 2024 20:07:47 +0000 Subject: [PATCH 31/54] time_virtual_col: initial --- .../ObjectStorage/StorageObjectStorageSource.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 13 +++++++++++-- src/Storages/VirtualColumnUtils.h | 2 +- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index b31d0f8a92e..8d5c03ae11e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -199,7 +199,7 @@ Chunk StorageObjectStorageSource::generate() chunk, read_from_format_info.requested_virtual_columns, getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), - object_info.metadata->size_bytes, &filename); + object_info.metadata->size_bytes, &filename, object_info.metadata->last_modified); return chunk; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index cec55cefda2..5362cffece5 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -111,7 +112,7 @@ void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) NameSet getVirtualNamesForFileLikeStorage() { - return {"_path", "_file", "_size"}; + return {"_path", "_file", "_size", "_time"}; } VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns) @@ -129,6 +130,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription add_virtual("_path", std::make_shared(std::make_shared())); add_virtual("_file", std::make_shared(std::make_shared())); add_virtual("_size", makeNullable(std::make_shared())); + add_virtual("_time", makeNullable(std::make_shared())); return desc; } @@ -188,7 +190,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const } void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename) + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename, std::optional last_modified) { for (const auto & virtual_column : requested_virtual_columns) { @@ -216,6 +218,13 @@ void addRequestedPathFileAndSizeVirtualsToChunk( else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } + else if (virtual_column.name == "_time") + { + if (last_modified) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), last_modified->utcTime())->convertToFullColumnIfConst()); + else + chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); + } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 62f2e4855b5..65826e1f1e7 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -69,7 +69,7 @@ void filterByPathOrFile(std::vector & sources, const std::vector & pa } void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr); + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr, std::optional last_modified = std::nullopt); } } From aefe7ffc31e4b2bb000d4df121472e85b93baa1c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 5 Jun 2024 21:34:29 +0000 Subject: [PATCH 32/54] time_virtual_col: slightly works --- src/IO/Archives/IArchiveReader.h | 2 ++ src/IO/Archives/LibArchiveReader.cpp | 1 + src/Storages/StorageFile.cpp | 4 +++- src/Storages/StorageFile.h | 1 + src/Storages/VirtualColumnUtils.cpp | 2 +- tests/integration/test_storage_hdfs/test.py | 2 +- tests/integration/test_storage_s3/test.py | 3 ++- 7 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/IO/Archives/IArchiveReader.h b/src/IO/Archives/IArchiveReader.h index ee516d2655b..d7758b9e401 100644 --- a/src/IO/Archives/IArchiveReader.h +++ b/src/IO/Archives/IArchiveReader.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -25,6 +26,7 @@ public: { UInt64 uncompressed_size; UInt64 compressed_size; + Poco::Timestamp last_modified; bool is_encrypted; }; diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index bec7f587180..e3fe63fa40d 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -157,6 +157,7 @@ public: file_info.emplace(); file_info->uncompressed_size = archive_entry_size(current_entry); file_info->compressed_size = archive_entry_size(current_entry); + file_info->last_modified = archive_entry_mtime(current_entry); file_info->is_encrypted = false; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 6744159d5dc..f84d3380c3f 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1341,6 +1341,7 @@ Chunk StorageFileSource::generate() chassert(file_enumerator); current_path = fmt::format("{}::{}", archive_reader->getPath(), *filename_override); current_file_size = file_enumerator->getFileInfo().uncompressed_size; + current_file_last_modified = file_enumerator->getFileInfo().last_modified; if (need_only_count && tryGetCountFromCache(current_archive_stat)) continue; @@ -1370,6 +1371,7 @@ Chunk StorageFileSource::generate() struct stat file_stat; file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); current_file_size = file_stat.st_size; + current_file_last_modified = Poco::Timestamp::fromEpochTime(file_stat.st_mtim.tv_sec); if (getContext()->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) continue; @@ -1437,7 +1439,7 @@ Chunk StorageFileSource::generate() /// Enrich with virtual columns. VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, current_path, current_file_size, filename_override.has_value() ? &filename_override.value() : nullptr); + chunk, requested_virtual_columns, current_path, current_file_size, filename_override.has_value() ? &filename_override.value() : nullptr, current_file_last_modified); return chunk; } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 37da59c3664..ac094aeb489 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -279,6 +279,7 @@ private: FilesIteratorPtr files_iterator; String current_path; std::optional current_file_size; + std::optional current_file_last_modified; struct stat current_archive_stat; std::optional filename_override; Block sample_block; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 5362cffece5..1e39d64fb18 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -221,7 +221,7 @@ void addRequestedPathFileAndSizeVirtualsToChunk( else if (virtual_column.name == "_time") { if (last_modified) - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), last_modified->utcTime())->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), last_modified->epochTime())->convertToFullColumnIfConst()); else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 44c0223e677..98ea79d6ee4 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -987,7 +987,7 @@ def test_read_subcolumns(started_cluster): assert res == "2\ttest_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( - f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select x.b.d, _path, x.b, _file, _time, now(), x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 09b27fff1e8..5becdf30476 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2117,7 +2117,7 @@ def test_read_subcolumns(started_cluster): assert res == "0\troot/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = instance.query( - f"select x.b.d, _path, x.b, _file, x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + f"select x.b.d, _path, x.b, _file, _time, now(), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" @@ -2148,6 +2148,7 @@ def test_read_subcolumns(started_cluster): res == "42\t/root/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" ) + logging.info("Some custom logging") def test_filtering_by_file_or_path(started_cluster): bucket = started_cluster.minio_bucket From 54ed3354f6be7aab304f55e253d8175a03d9a2b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 12:56:43 +0200 Subject: [PATCH 33/54] Better debugging --- ...23_zeros_generate_random_with_limit_progress_bar.sh | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh index 4bb8fc8880d..8aedf0bc0ff 100755 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -6,11 +7,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_with_progress_and_match_total_rows() { - echo "$1" | \ - ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1 | \ - grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found" + CURL_RESPONSE=$(echo "$1" | \ + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) + + echo "$CURL_RESPONSE" | grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found: ${CURL_RESPONSE}" } run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros LIMIT 100' run_with_progress_and_match_total_rows 'SELECT * FROM system.zeros_mt LIMIT 100' -run_with_progress_and_match_total_rows 'SELECT * FROM generateRandom() LIMIT 100' +run_with_progress_and_match_total_rows "SELECT * FROM generateRandom('number UInt64') LIMIT 100" From fab23d2a16573e3a9613cb53c2aba4639f8e33eb Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 6 Jun 2024 15:02:33 +0200 Subject: [PATCH 34/54] Add lowCardinalityKeys, lowCardinalityIndices and an example for blockSize --- .../functions/other-functions.md | 149 +++++++++++++++++- 1 file changed, 148 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 31df9e5627d..4639f9d0991 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -212,7 +212,7 @@ toTypeName(x) ## blockSize {#blockSize} -In ClickHouse, queries are processed in blocks (chunks). +In ClickHouse, queries are processed in [blocks](../../development/architecture.md/#block-block) (chunks). This function returns the size (row count) of the block the function is called on. **Syntax** @@ -221,6 +221,38 @@ This function returns the size (row count) of the block the function is called o blockSize() ``` +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (n UInt8) ENGINE = Memory; +-- Insert 3 blocks: +INSERT INTO test VALUES (1); +INSERT INTO test VALUES (1),(2); +INSERT INTO test VALUES (1),(2),(3); + +SELECT blockSize(), n FROM (SELECT * FROM test); +``` + +Result: + +```response + ┌─blockSize()─┬─n─┐ +1. │ 1 │ 1 │ + └─────────────┴───┘ + ┌─blockSize()─┬─n─┐ +2. │ 3 │ 1 │ +3. │ 3 │ 2 │ +4. │ 3 │ 3 │ + └─────────────┴───┘ + ┌─blockSize()─┬─n─┐ +5. │ 2 │ 1 │ +6. │ 2 │ 2 │ + └─────────────┴───┘ +``` + ## byteSize Returns an estimation of uncompressed byte size of its arguments in memory. @@ -3688,3 +3720,118 @@ Result: ```response {'version':'1','serial_number':'2D9071D64530052D48308473922C7ADAFA85D6C5','signature_algo':'sha256WithRSAEncryption','issuer':'/CN=marsnet.local CA','not_before':'May 7 17:01:21 2024 GMT','not_after':'May 7 17:01:21 2025 GMT','subject':'/CN=chnode1','pkey_algo':'rsaEncryption'} ``` + +## lowCardinalityIndices + +For each row in the current [block](../../development/architecture.md/#block-block), returns the index of the value in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. + +:::note +The first unique value encountered in the block is enumerated from 1. +::: + +**Syntax** + +```sql +lowCardinalityIndices(col) +``` + +**Arguments** + +- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md). + +**Returned value** + +- returns the index of the value in the dictionary of unique values, for each row in the current block. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; + +-- insert two blocks of data: + +INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); +INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); + +SELECT s, lowCardinalityIndices(s) FROM test; +``` + +Result: + +```response + ┌─s───┬─lowCardinalityIndices(s)─┐ +1. │ one │ 1 │ +2. │ two │ 2 │ +3. │ one │ 1 │ +4. │ one │ 1 │ +5. │ two │ 2 │ + └─────┴──────────────────────────┘ + ┌─s─────┬─lowCardinalityIndices(s)─┐ + 6. │ three │ 1 │ + 7. │ two │ 2 │ + 8. │ one │ 3 │ + 9. │ two │ 2 │ +10. │ two │ 2 │ +11. │ three │ 1 │ + └───────┴──────────────────────────┘ +``` +## lowCardinalityKeys + +For each row in the current [block](../../development/architecture.md/#block-block), returns the keys (unique values) in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. + +:::note +If the column size is less than the dictionary size, then values will be cut. If it is greater, then defaults will be added. +::: + +**Syntax** + +```sql +lowCardinalityIndices(col) +``` + +**Arguments** + +- `col` — a low cardinality column. [LowCardinality](../data-types/lowcardinality.md). + +**Returned value** + +- returns the keys of the dictionary, for each row in the current block. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test; +CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; + +-- insert two blocks of data: + +INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); +INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); + +SELECT s, lowCardinalityKeys(s) FROM test; +``` + +Result: + +```response + ┌─s───┬─lowCardinalityKeys(s)─┐ +1. │ one │ │ +2. │ two │ one │ +3. │ one │ two │ +4. │ one │ │ +5. │ two │ │ + └─────┴───────────────────────┘ + ┌─s─────┬─lowCardinalityKeys(s)─┐ + 6. │ three │ │ + 7. │ two │ three │ + 8. │ one │ two │ + 9. │ two │ one │ +10. │ two │ │ +11. │ three │ │ + └───────┴───────────────────────┘ +``` From 387869488d80db6c645346c61b2470d37a026fe6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 15:54:02 +0200 Subject: [PATCH 35/54] Stabilize and adjust number of streams --- src/Storages/StorageGenerateRandom.cpp | 58 +++++++++++++++---- src/Storages/System/StorageSystemZeros.cpp | 33 ++++++----- ...generate_random_with_limit_progress_bar.sh | 2 +- 3 files changed, 65 insertions(+), 28 deletions(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index cdbade51695..5aceef78238 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -50,6 +50,12 @@ namespace ErrorCodes namespace { +struct GenerateRandomState +{ + std::atomic add_total_rows = 0; +}; +using GenerateRandomStatePtr = std::shared_ptr; + void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_of_type, pcg64 & rng, [[maybe_unused]] bool flip_bytes = false) { size_t size = limit * size_of_type; @@ -529,10 +535,24 @@ ColumnPtr fillColumnWithRandomData( class GenerateSource : public ISource { public: - GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_) + GenerateSource( + UInt64 block_size_, + UInt64 max_array_length_, + UInt64 max_string_length_, + UInt64 random_seed_, + Block block_header_, + ContextPtr context_, + GenerateRandomStatePtr state_) : ISource(Nested::flattenNested(prepareBlockToFill(block_header_))) - , block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_) - , block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {} + , block_size(block_size_) + , max_array_length(max_array_length_) + , max_string_length(max_string_length_) + , block_to_fill(std::move(block_header_)) + , rng(random_seed_) + , context(context_) + , shared_state(state_) + { + } String getName() const override { return "GenerateRandom"; } @@ -546,7 +566,15 @@ protected: columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context)); columns = Nested::flattenNested(block_to_fill.cloneWithColumns(columns)).getColumns(); - return {std::move(columns), block_size}; + + UInt64 total_rows = shared_state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + + auto chunk = Chunk{std::move(columns), block_size}; + progress(chunk.getNumRows(), chunk.bytes()); + + return chunk; } private: @@ -558,6 +586,7 @@ private: pcg64 rng; ContextPtr context; + GenerateRandomStatePtr shared_state; static Block & prepareBlockToFill(Block & block) { @@ -645,9 +674,6 @@ Pipe StorageGenerateRandom::read( { storage_snapshot->check(column_names); - Pipes pipes; - pipes.reserve(num_streams); - const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns(); Block block_header; for (const auto & name : column_names) @@ -676,16 +702,24 @@ Pipe StorageGenerateRandom::read( } } + UInt64 query_limit = query_info.limit; + if (query_limit && num_streams * max_block_size < query_limit) + { + /// We want to avoid spawning more streams than necessary + num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + } + Pipes pipes; + pipes.reserve(num_streams); + /// Will create more seed values for each source from initial seed. pcg64 generate(random_seed); + auto shared_state = std::make_shared(query_info.limit); + for (UInt64 i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context); - - if (i == 0 && query_info.limit) - source->addTotalRowsApprox(query_info.limit); - + auto source = std::make_shared( + max_block_size, max_array_length, max_string_length, generate(), block_header, context, shared_state); pipes.emplace_back(std::move(source)); } diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index a48b109fbbe..40faf2e265f 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -16,7 +16,9 @@ namespace struct ZerosState { + ZerosState(UInt64 limit) : add_total_rows(limit) { } std::atomic num_generated_rows = 0; + std::atomic add_total_rows = 0; }; using ZerosStatePtr = std::shared_ptr; @@ -49,6 +51,10 @@ protected: if (generated_rows >= limit) return {}; + UInt64 total_rows = state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + if (generated_rows + column_size > limit) { column_size = limit - generated_rows; @@ -105,10 +111,16 @@ Pipe StorageSystemZeros::read( bool use_multiple_streams = multithreaded; - if (limit && *limit < max_block_size) + UInt64 query_limit = limit ? *limit : query_info.limit; + if (query_limit && query_limit > max_block_size) + max_block_size = query_limit; + + if (use_multiple_streams && query_limit && num_streams * max_block_size < query_limit) { - max_block_size = static_cast(*limit); - use_multiple_streams = false; + /// We want to avoid spawning more streams than necessary + num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + if (num_streams <= 1) + use_multiple_streams = false; } if (!use_multiple_streams) @@ -118,21 +130,12 @@ Pipe StorageSystemZeros::read( ZerosStatePtr state; - if (limit) - state = std::make_shared(); + if (query_limit) + state = std::make_shared(query_limit); for (size_t i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, limit ? *limit : 0, state); - - if (i == 0) - { - if (limit) - source->addTotalRowsApprox(*limit); - else if (query_info.limit) - source->addTotalRowsApprox(query_info.limit); - } - + auto source = std::make_shared(max_block_size, query_limit, state); res.addSource(std::move(source)); } diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh index 8aedf0bc0ff..500a12587a2 100755 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function run_with_progress_and_match_total_rows() { CURL_RESPONSE=$(echo "$1" | \ - ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) echo "$CURL_RESPONSE" | grep -q '"total_rows_to_read":"100"' && echo "Matched" || echo "Expected total_rows_to_read not found: ${CURL_RESPONSE}" } From 4ba025c3ba8ceaa1551670f29b328983741f8052 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 15:48:36 +0000 Subject: [PATCH 36/54] Fix duplicating Delete events in blob_storage_log --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index ae719f5cde4..afc13251f5b 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -382,6 +382,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { std::vector current_chunk; String keys; + size_t first_position = current_position; for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position) { Aws::S3::Model::ObjectIdentifier obj; @@ -407,9 +408,9 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError(); auto time_now = std::chrono::system_clock::now(); - for (const auto & object : objects) + for (size_t i = first_position; i < current_position; ++i) blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, - uri.bucket, object.remote_path, object.local_path, object.bytes_size, + uri.bucket, objects[i].remote_path, objects[i].local_path, objects[i].bytes_size, outcome_error, time_now); } From 38d45a6bc99e738a051ccb386e3c0db706051c32 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 15:49:14 +0000 Subject: [PATCH 37/54] Verbose message for errors on inserting to system log --- src/Interpreters/SystemLog.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 5e0ce2cb0de..24e478053bc 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -499,7 +499,10 @@ void SystemLog::flushImpl(const std::vector & to_flush, auto log_element_names_and_types = LogElement::getColumnsDescription(); for (const auto & name_and_type : log_element_names_and_types.getAll()) - log_element_columns.emplace_back(name_and_type.type, name_and_type.name); + { + auto & column = log_element_columns.emplace_back(name_and_type.type, name_and_type.name); + column.column->reserve(to_flush.size()); + } Block block(std::move(log_element_columns)); @@ -532,7 +535,8 @@ void SystemLog::flushImpl(const std::vector & to_flush, } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to flush system log {} with {} entries up to offset {}", + table_id.getNameForLogs(), to_flush.size(), to_flush_end)); } queue->confirm(to_flush_end); From 81fe4c48944eda05c8aad628e0de65b47aeddead Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 17:59:16 +0200 Subject: [PATCH 38/54] Fix number of stream decision --- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/System/StorageSystemZeros.cpp | 35 ++++++++-------------- 2 files changed, 14 insertions(+), 23 deletions(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 5aceef78238..ca9c6fb3226 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -703,7 +703,7 @@ Pipe StorageGenerateRandom::read( } UInt64 query_limit = query_info.limit; - if (query_limit && num_streams * max_block_size < query_limit) + if (query_limit && num_streams * max_block_size > query_limit) { /// We want to avoid spawning more streams than necessary num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 40faf2e265f..7e1d7a14d0e 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -44,17 +44,16 @@ protected: auto column_ptr = column; size_t column_size = column_ptr->size(); - if (state) + UInt64 total_rows = state->add_total_rows.fetch_and(0); + if (total_rows) + addTotalRowsApprox(total_rows); + + if (limit) { auto generated_rows = state->num_generated_rows.fetch_add(column_size, std::memory_order_acquire); - if (generated_rows >= limit) return {}; - UInt64 total_rows = state->add_total_rows.fetch_and(0); - if (total_rows) - addTotalRowsApprox(total_rows); - if (generated_rows + column_size > limit) { column_size = limit - generated_rows; @@ -109,30 +108,22 @@ Pipe StorageSystemZeros::read( { storage_snapshot->check(column_names); - bool use_multiple_streams = multithreaded; + UInt64 query_limit = limit ? *limit : 0; + if (query_info.limit) + query_limit = query_limit ? std::min(query_limit, query_info.limit) : query_info.limit; - UInt64 query_limit = limit ? *limit : query_info.limit; - if (query_limit && query_limit > max_block_size) + if (query_limit && query_limit < max_block_size) max_block_size = query_limit; - if (use_multiple_streams && query_limit && num_streams * max_block_size < query_limit) - { + if (!multithreaded) + num_streams = 1; + else if (query_limit && num_streams * max_block_size > query_limit) /// We want to avoid spawning more streams than necessary num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); - if (num_streams <= 1) - use_multiple_streams = false; - } - if (!use_multiple_streams) - num_streams = 1; + ZerosStatePtr state = std::make_shared(query_limit); Pipe res; - - ZerosStatePtr state; - - if (query_limit) - state = std::make_shared(query_limit); - for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared(max_block_size, query_limit, state); From 1881107f0773d4aa0e195ef5cd6bb8f8a17b8cc3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 16:04:46 +0000 Subject: [PATCH 39/54] fix build --- src/Interpreters/SystemLog.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 24e478053bc..3b25deeb59d 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -499,14 +499,15 @@ void SystemLog::flushImpl(const std::vector & to_flush, auto log_element_names_and_types = LogElement::getColumnsDescription(); for (const auto & name_and_type : log_element_names_and_types.getAll()) - { - auto & column = log_element_columns.emplace_back(name_and_type.type, name_and_type.name); - column.column->reserve(to_flush.size()); - } + log_element_columns.emplace_back(name_and_type.type, name_and_type.name); Block block(std::move(log_element_columns)); MutableColumns columns = block.mutateColumns(); + + for (auto & column : columns) + column->reserve(to_flush.size()); + for (const auto & elem : to_flush) elem.appendToBlock(columns); From d36bf5bac86fb9bfde4fe761a1ab2ab7079f4f1a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 6 Jun 2024 21:40:24 +0200 Subject: [PATCH 40/54] Compiler complains --- src/Storages/System/StorageSystemZeros.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index 7e1d7a14d0e..09a2bb5d963 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -16,7 +16,7 @@ namespace struct ZerosState { - ZerosState(UInt64 limit) : add_total_rows(limit) { } + explicit ZerosState(UInt64 limit) : add_total_rows(limit) { } std::atomic num_generated_rows = 0; std::atomic add_total_rows = 0; }; @@ -119,7 +119,7 @@ Pipe StorageSystemZeros::read( num_streams = 1; else if (query_limit && num_streams * max_block_size > query_limit) /// We want to avoid spawning more streams than necessary - num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + num_streams = std::min(num_streams, static_cast(((query_limit + max_block_size - 1) / max_block_size))); ZerosStatePtr state = std::make_shared(query_limit); From 258b1f9559673b3ebe6da0eae728aa783955045e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 6 Jun 2024 21:00:47 +0000 Subject: [PATCH 41/54] time_virtual_col: tests, doc, small refactoring --- .../integrations/azureBlobStorage.md | 1 + .../table-engines/integrations/hdfs.md | 1 + .../engines/table-engines/integrations/s3.md | 11 +++++---- docs/en/engines/table-engines/special/file.md | 1 + docs/en/engines/table-engines/special/url.md | 1 + .../table-functions/azureBlobStorage.md | 1 + docs/en/sql-reference/table-functions/file.md | 1 + docs/en/sql-reference/table-functions/hdfs.md | 1 + docs/en/sql-reference/table-functions/s3.md | 1 + docs/en/sql-reference/table-functions/url.md | 1 + .../StorageObjectStorageSource.cpp | 14 ++++++----- src/Storages/S3Queue/S3QueueSource.cpp | 10 ++++++-- src/Storages/StorageFile.cpp | 11 +++++++-- src/Storages/StorageURL.cpp | 7 +++++- src/Storages/VirtualColumnUtils.cpp | 23 ++++++++++--------- src/Storages/VirtualColumnUtils.h | 14 +++++++++-- .../test_storage_azure_blob_storage/test.py | 4 ++-- tests/integration/test_storage_hdfs/test.py | 4 ++-- tests/integration/test_storage_s3/test.py | 4 ++-- ...e_structure_from_insertion_table.reference | 1 - ...lumn_use_structure_from_insertion_table.sh | 13 ----------- ...e_structure_from_insertion_table.reference | 1 + ...lumn_use_structure_from_insertion_table.sh | 14 +++++++++++ 23 files changed, 91 insertions(+), 49 deletions(-) delete mode 100644 tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference delete mode 100755 tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh create mode 100644 tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference create mode 100755 tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index 0843ff1ac47..dfc27d6b8cf 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -54,6 +54,7 @@ SELECT * FROM test_table; - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## See also diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 2749fa7e479..c9df713231a 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -235,6 +235,7 @@ libhdfs3 support HDFS namenode HA. - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index cb1da1c8e68..93f4a187656 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -53,14 +53,14 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da This example uses the [docker compose recipe](https://github.com/ClickHouse/examples/tree/5fdc6ff72f4e5137e23ea075c88d3f44b0202490/docker-compose-recipes/recipes/ch-and-minio-S3), which integrates ClickHouse and MinIO. You should be able to reproduce the same queries using S3 by replacing the endpoint and authentication values. -Notice that the S3 endpoint in the `ENGINE` configuration uses the parameter token `{_partition_id}` as part of the S3 object (filename), and that the SELECT queries select against those resulting object names (e.g., `test_3.csv`). +Notice that the S3 endpoint in the `ENGINE` configuration uses the parameter token `{_partition_id}` as part of the S3 object (filename), and that the SELECT queries select against those resulting object names (e.g., `test_3.csv`). :::note As shown in the example, querying from S3 tables that are partitioned is not directly supported at this time, but can be accomplished by querying the individual partitions using the S3 table function. -The primary use-case for writing +The primary use-case for writing partitioned data in S3 is to enable transferring that data into another ClickHouse system (for example, moving from on-prem systems to ClickHouse Cloud). Because ClickHouse datasets are often very large, and network @@ -78,9 +78,9 @@ CREATE TABLE p ) ENGINE = S3( # highlight-next-line - 'http://minio:10000/clickhouse//test_{_partition_id}.csv', - 'minioadmin', - 'minioadminpassword', + 'http://minio:10000/clickhouse//test_{_partition_id}.csv', + 'minioadmin', + 'minioadminpassword', 'CSV') PARTITION BY column3 ``` @@ -145,6 +145,7 @@ Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Reading fr - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index 0d422f64762..957b18b5305 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -102,6 +102,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Settings {#settings} diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index f6183a779ae..c906830d0e9 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -108,6 +108,7 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da - `_path` — Path to the `URL`. Type: `LowCardinalty(String)`. - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 1510489ce83..f59fedeb3a2 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -72,6 +72,7 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. **See Also** diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index f66178afbb2..4fec772c373 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -196,6 +196,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Settings {#settings} diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index d65615e7588..28cba5ccc6a 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -97,6 +97,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index cbef80371a3..1a7e2b8d66a 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -272,6 +272,7 @@ FROM s3( - `_path` — Path to the file. Type: `LowCardinalty(String)`. - `_file` — Name of the file. Type: `LowCardinalty(String)`. - `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 4dc6e435b50..3bb7aff53a7 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -53,6 +53,7 @@ Character `|` inside patterns is used to specify failover addresses. They are it - `_path` — Path to the `URL`. Type: `LowCardinalty(String)`. - `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. - `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. +- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 8d5c03ae11e..2fc6993369d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -195,12 +195,14 @@ Chunk StorageObjectStorageSource::generate() const auto & object_info = reader.getObjectInfo(); const auto & filename = object_info.getFileName(); chassert(object_info.metadata); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, - read_from_format_info.requested_virtual_columns, - getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), - object_info.metadata->size_bytes, &filename, object_info.metadata->last_modified); - + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, read_from_format_info.requested_virtual_columns, + { + .path = getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), + .size = object_info.metadata->size_bytes, + .filename = &filename, + .last_modified = object_info.metadata->last_modified + }); return chunk; } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index d8633037ed9..b5b1a8dd992 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -421,8 +421,14 @@ Chunk StorageS3QueueSource::generate() file_status->processed_rows += chunk.getNumRows(); processed_rows_from_file += chunk.getNumRows(); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, path, reader.getObjectInfo().metadata->size_bytes); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = path, + .size = reader.getObjectInfo().metadata->size_bytes + }); + + return chunk; } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index f84d3380c3f..09f0bd60859 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1438,8 +1438,15 @@ Chunk StorageFileSource::generate() progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); /// Enrich with virtual columns. - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, current_path, current_file_size, filename_override.has_value() ? &filename_override.value() : nullptr, current_file_last_modified); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = current_path, + .size = current_file_size, + .filename = (filename_override.has_value() ? &filename_override.value() : nullptr), + .last_modified = current_file_last_modified + }); + return chunk; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 8d1c6933503..9302e7ef3e5 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -411,7 +411,12 @@ Chunk StorageURLSource::generate() if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, curr_uri.getPath(), current_file_size); + VirtualColumnUtils::addRequestedFileLikeStorageVirtualsToChunk( + chunk, requested_virtual_columns, + { + .path = curr_uri.getPath(), + .size = current_file_size + }); return chunk; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 1e39d64fb18..778c9e13adb 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -189,39 +189,40 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } -void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename, std::optional last_modified) +void addRequestedFileLikeStorageVirtualsToChunk( + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, + VirtualsForFileLikeStorage virtual_values) { for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") { - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), path)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), virtual_values.path)->convertToFullColumnIfConst()); } else if (virtual_column.name == "_file") { - if (filename) + if (virtual_values.filename) { - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *filename)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), (*virtual_values.filename))->convertToFullColumnIfConst()); } else { - size_t last_slash_pos = path.find_last_of('/'); - auto filename_from_path = path.substr(last_slash_pos + 1); + size_t last_slash_pos = virtual_values.path.find_last_of('/'); + auto filename_from_path = virtual_values.path.substr(last_slash_pos + 1); chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), filename_from_path)->convertToFullColumnIfConst()); } } else if (virtual_column.name == "_size") { - if (size) - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *size)->convertToFullColumnIfConst()); + if (virtual_values.size) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *virtual_values.size)->convertToFullColumnIfConst()); else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } else if (virtual_column.name == "_time") { - if (last_modified) - chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), last_modified->epochTime())->convertToFullColumnIfConst()); + if (virtual_values.last_modified) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), virtual_values.last_modified->epochTime())->convertToFullColumnIfConst()); else chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 65826e1f1e7..fbfbdd6c6cc 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -68,8 +68,18 @@ void filterByPathOrFile(std::vector & sources, const std::vector & pa sources = std::move(filtered_sources); } -void addRequestedPathFileAndSizeVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr, std::optional last_modified = std::nullopt); +struct VirtualsForFileLikeStorage +{ + const String & path; + std::optional size { std::nullopt }; + const String * filename { nullptr }; + std::optional last_modified { std::nullopt }; + +}; + +void addRequestedFileLikeStorageVirtualsToChunk( + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, + VirtualsForFileLikeStorage virtual_values); } } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f836c58ce30..9f5aef1489c 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -758,12 +758,12 @@ def test_read_subcolumns(cluster): ) res = node.query( - f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv'," + f"select a.b.d, _path, a.b, _file, dateDiff('minute', _time, now()), a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv'," f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" + assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t0\t3\n" res = node.query( f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 98ea79d6ee4..cda2b8694c6 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -987,10 +987,10 @@ def test_read_subcolumns(started_cluster): assert res == "2\ttest_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( - f"select x.b.d, _path, x.b, _file, _time, now(), x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select x.b.d, _path, x.b, _file, dateDiff('minute', _time, now()), x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" + assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\t0\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 5becdf30476..f7f7bbe335c 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2117,10 +2117,10 @@ def test_read_subcolumns(started_cluster): assert res == "0\troot/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = instance.query( - f"select x.b.d, _path, x.b, _file, _time, now(), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + f"select x.b.d, _path, x.b, _file, dateDiff('minute', _time, now()), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) - assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" + assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t0\t42\n" res = instance.query( f"select a.b.d, _path, a.b, _file, a.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" diff --git a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference deleted file mode 100644 index 35ef86f5339..00000000000 --- a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.reference +++ /dev/null @@ -1 +0,0 @@ -1 2 4 diff --git a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh b/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh deleted file mode 100755 index d9e4a2c8f8b..00000000000 --- a/tests/queries/0_stateless/02931_size_virtual_column_use_structure_from_insertion_table.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv -$CLICKHOUSE_LOCAL -nm -q " -create table test (x UInt64, y UInt32, size UInt64) engine=Memory; -insert into test select c1, c2, _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') settings use_structure_from_insertion_table_in_table_functions=1; -select * from test; -" -rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv diff --git a/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference new file mode 100644 index 00000000000..93acdc34842 --- /dev/null +++ b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.reference @@ -0,0 +1 @@ +1 2 4 1 1 diff --git a/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh new file mode 100755 index 00000000000..ebdda0cc1d3 --- /dev/null +++ b/tests/queries/0_stateless/02931_virtual_column_use_structure_from_insertion_table.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "1,2" > $CLICKHOUSE_TEST_UNIQUE_NAME.csv +sleep 1 +$CLICKHOUSE_LOCAL -nm -q " +create table test (x UInt64, y UInt32, size UInt64, d32 DateTime32, d64 DateTime64) engine=Memory; +insert into test select c1, c2, _size, _time, _time from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') settings use_structure_from_insertion_table_in_table_functions=1; +select x, y, size, (dateDiff('millisecond', d32, now()) < 4000 AND dateDiff('millisecond', d32, now()) > 0), (dateDiff('second', d64, now()) < 4 AND dateDiff('second', d64, now()) > 0) from test; +" +rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv From b82ab5a4028f620d4abe920745945983ae9f264c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 6 Jun 2024 21:21:12 +0000 Subject: [PATCH 42/54] time_virtual_col: style check --- tests/integration/test_storage_s3/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index f7f7bbe335c..61c6d95f123 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2120,7 +2120,9 @@ def test_read_subcolumns(started_cluster): f"select x.b.d, _path, x.b, _file, dateDiff('minute', _time, now()), x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) - assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t0\t42\n" + assert ( + res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t0\t42\n" + ) res = instance.query( f"select a.b.d, _path, a.b, _file, a.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" @@ -2150,6 +2152,7 @@ def test_read_subcolumns(started_cluster): logging.info("Some custom logging") + def test_filtering_by_file_or_path(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] From 47578c7243c4379fea1677e32e0e3a2733d8b074 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 7 Jun 2024 08:43:49 +0200 Subject: [PATCH 43/54] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 49f43615c7e..826c6953a10 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1934,6 +1934,8 @@ london lookups loongarch lowcardinality +lowCardinalityIndices +lowCardinalityKeys lowerUTF lowercased lttb From 078f5f4ee0fff1d55a5924df8340a905ef32f0f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 13:10:12 +0200 Subject: [PATCH 44/54] Fix bug in short circuit evaluation --- src/Columns/MaskOperations.cpp | 6 +++++- .../0_stateless/03168_fuzz_multiIf_short_circuit.reference | 0 .../0_stateless/03168_fuzz_multiIf_short_circuit.sql | 6 ++++++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference create mode 100644 tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index 1f5f94beee9..873a4060872 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -289,10 +289,14 @@ void executeColumnIfNeeded(ColumnWithTypeAndName & column, bool empty) if (!column_function) return; + size_t original_size = column.column->size(); + if (!empty) column = column_function->reduce(); else - column.column = column_function->getResultType()->createColumn(); + column.column = column_function->getResultType()->createColumnConstWithDefaultValue(original_size)->convertToFullColumnIfConst(); + + chassert(column.column->size() == original_size); } int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments) diff --git a/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql new file mode 100644 index 00000000000..4e4cc291e9b --- /dev/null +++ b/tests/queries/0_stateless/03168_fuzz_multiIf_short_circuit.sql @@ -0,0 +1,6 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/64946 +SELECT + multiIf((number % toLowCardinality(toNullable(toUInt128(2)))) = (number % toNullable(2)), toInt8(1), (number % materialize(toLowCardinality(3))) = toUInt128(toNullable(0)), toInt8(materialize(materialize(2))), toInt64(toUInt128(3))) +FROM system.numbers +LIMIT 44857 +FORMAT Null; From 4b010dc478310b65d26cbe114e15f3cb73af4bb4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 7 Jun 2024 13:11:52 +0200 Subject: [PATCH 45/54] Disable test with ASAN --- .../0_stateless/02908_many_requests_to_system_replicas.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh index 144831a2cdc..a247c99a818 100755 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-parallel, no-fasttest, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 4d88f103469e8176229f3a258847eb9cca374309 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 7 Jun 2024 11:28:36 +0000 Subject: [PATCH 46/54] Update version_date.tsv and changelogs after v24.4.2.141-stable --- docs/changelogs/v24.4.2.141-stable.md | 101 ++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 102 insertions(+) create mode 100644 docs/changelogs/v24.4.2.141-stable.md diff --git a/docs/changelogs/v24.4.2.141-stable.md b/docs/changelogs/v24.4.2.141-stable.md new file mode 100644 index 00000000000..656d0854392 --- /dev/null +++ b/docs/changelogs/v24.4.2.141-stable.md @@ -0,0 +1,101 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.4.2.141-stable (9e23d27bd11) FIXME as compared to v24.4.1.2088-stable (6d4b31322d1) + +#### Improvement +* Backported in [#63467](https://github.com/ClickHouse/ClickHouse/issues/63467): Make rabbitmq nack broken messages. Closes [#45350](https://github.com/ClickHouse/ClickHouse/issues/45350). [#60312](https://github.com/ClickHouse/ClickHouse/pull/60312) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Build/Testing/Packaging Improvement +* Backported in [#63612](https://github.com/ClickHouse/ClickHouse/issues/63612): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#64279](https://github.com/ClickHouse/ClickHouse/issues/64279): Fix queries with FINAL give wrong result when table does not use adaptive granularity. [#62432](https://github.com/ClickHouse/ClickHouse/pull/62432) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#63295](https://github.com/ClickHouse/ClickHouse/issues/63295): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63978](https://github.com/ClickHouse/ClickHouse/issues/63978): Fix intersect parts when restart after drop range. [#63202](https://github.com/ClickHouse/ClickHouse/pull/63202) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#63413](https://github.com/ClickHouse/ClickHouse/issues/63413): Fix a misbehavior when SQL security defaults don't load for old tables during server startup. [#63209](https://github.com/ClickHouse/ClickHouse/pull/63209) ([pufit](https://github.com/pufit)). +* Backported in [#63388](https://github.com/ClickHouse/ClickHouse/issues/63388): JOIN filter push down filled join fix. Closes [#63228](https://github.com/ClickHouse/ClickHouse/issues/63228). [#63234](https://github.com/ClickHouse/ClickHouse/pull/63234) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63618](https://github.com/ClickHouse/ClickHouse/issues/63618): Fix bug which could potentially lead to rare LOGICAL_ERROR during SELECT query with message: `Unexpected return type from materialize. Expected type_XXX. Got type_YYY.` Introduced in [#59379](https://github.com/ClickHouse/ClickHouse/issues/59379). [#63353](https://github.com/ClickHouse/ClickHouse/pull/63353) ([alesapin](https://github.com/alesapin)). +* Backported in [#63451](https://github.com/ClickHouse/ClickHouse/issues/63451): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)). +* Backported in [#63605](https://github.com/ClickHouse/ClickHouse/issues/63605): Fix backup of projection part in case projection was removed from table metadata, but part still has projection. [#63426](https://github.com/ClickHouse/ClickHouse/pull/63426) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#63510](https://github.com/ClickHouse/ClickHouse/issues/63510): Fix 'Every derived table must have its own alias' error for MYSQL dictionary source, close [#63341](https://github.com/ClickHouse/ClickHouse/issues/63341). [#63481](https://github.com/ClickHouse/ClickHouse/pull/63481) ([vdimir](https://github.com/vdimir)). +* Backported in [#63592](https://github.com/ClickHouse/ClickHouse/issues/63592): Avoid segafult in `MergeTreePrefetchedReadPool` while fetching projection parts. [#63513](https://github.com/ClickHouse/ClickHouse/pull/63513) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#63750](https://github.com/ClickHouse/ClickHouse/issues/63750): Read only the necessary columns from VIEW (new analyzer). Closes [#62594](https://github.com/ClickHouse/ClickHouse/issues/62594). [#63688](https://github.com/ClickHouse/ClickHouse/pull/63688) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63772](https://github.com/ClickHouse/ClickHouse/issues/63772): Fix [#63539](https://github.com/ClickHouse/ClickHouse/issues/63539). Forbid WINDOW redefinition in new analyzer. [#63694](https://github.com/ClickHouse/ClickHouse/pull/63694) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#63872](https://github.com/ClickHouse/ClickHouse/issues/63872): Flatten_nested is broken with replicated database. [#63695](https://github.com/ClickHouse/ClickHouse/pull/63695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63854](https://github.com/ClickHouse/ClickHouse/issues/63854): Fix `Not found column` and `CAST AS Map from array requires nested tuple of 2 elements` exceptions for distributed queries which use `Map(Nothing, Nothing)` type. Fixes [#63637](https://github.com/ClickHouse/ClickHouse/issues/63637). [#63753](https://github.com/ClickHouse/ClickHouse/pull/63753) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#63847](https://github.com/ClickHouse/ClickHouse/issues/63847): Fix possible `ILLEGAL_COLUMN` error in `partial_merge` join, close [#37928](https://github.com/ClickHouse/ClickHouse/issues/37928). [#63755](https://github.com/ClickHouse/ClickHouse/pull/63755) ([vdimir](https://github.com/vdimir)). +* Backported in [#63908](https://github.com/ClickHouse/ClickHouse/issues/63908): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#63955](https://github.com/ClickHouse/ClickHouse/issues/63955): Fix possible crash with SYSTEM UNLOAD PRIMARY KEY. [#63778](https://github.com/ClickHouse/ClickHouse/pull/63778) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63938](https://github.com/ClickHouse/ClickHouse/issues/63938): Allow JOIN filter push down to both streams if only single equivalent column is used in query. Closes [#63799](https://github.com/ClickHouse/ClickHouse/issues/63799). [#63819](https://github.com/ClickHouse/ClickHouse/pull/63819) ([Maksim Kita](https://github.com/kitaisreal)). +* Backported in [#63991](https://github.com/ClickHouse/ClickHouse/issues/63991): Fix incorrect select query result when parallel replicas were used to read from a Materialized View. [#63861](https://github.com/ClickHouse/ClickHouse/pull/63861) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#64033](https://github.com/ClickHouse/ClickHouse/issues/64033): Fix a error `Database name is empty` for remote queries with lambdas over the cluster with modified default database. Fixes [#63471](https://github.com/ClickHouse/ClickHouse/issues/63471). [#63864](https://github.com/ClickHouse/ClickHouse/pull/63864) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64561](https://github.com/ClickHouse/ClickHouse/issues/64561): Fix SIGSEGV due to CPU/Real (`query_profiler_real_time_period_ns`/`query_profiler_cpu_time_period_ns`) profiler (has been an issue since 2022, that leads to periodic server crashes, especially if you were using distributed engine). [#63865](https://github.com/ClickHouse/ClickHouse/pull/63865) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#64011](https://github.com/ClickHouse/ClickHouse/issues/64011): Fix analyzer - IN function with arbitrary deep sub-selects in materialized view to use insertion block. [#63930](https://github.com/ClickHouse/ClickHouse/pull/63930) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#64238](https://github.com/ClickHouse/ClickHouse/issues/64238): Fix resolve of unqualified COLUMNS matcher. Preserve the input columns order and forbid usage of unknown identifiers. [#63962](https://github.com/ClickHouse/ClickHouse/pull/63962) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64103](https://github.com/ClickHouse/ClickHouse/issues/64103): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64170](https://github.com/ClickHouse/ClickHouse/issues/64170): Add missing settings to recoverLostReplica. [#64040](https://github.com/ClickHouse/ClickHouse/pull/64040) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64322](https://github.com/ClickHouse/ClickHouse/issues/64322): This fix will use a proper redefined context with the correct definer for each individual view in the query pipeline Closes [#63777](https://github.com/ClickHouse/ClickHouse/issues/63777). [#64079](https://github.com/ClickHouse/ClickHouse/pull/64079) ([pufit](https://github.com/pufit)). +* Backported in [#64382](https://github.com/ClickHouse/ClickHouse/issues/64382): Fix analyzer: "Not found column" error is fixed when using INTERPOLATE. [#64096](https://github.com/ClickHouse/ClickHouse/pull/64096) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#64568](https://github.com/ClickHouse/ClickHouse/issues/64568): Fix creating backups to S3 buckets with different credentials from the disk containing the file. [#64153](https://github.com/ClickHouse/ClickHouse/pull/64153) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#64272](https://github.com/ClickHouse/ClickHouse/issues/64272): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64330](https://github.com/ClickHouse/ClickHouse/issues/64330): The query cache now considers two identical queries against different databases as different. The previous behavior could be used to bypass missing privileges to read from a table. [#64199](https://github.com/ClickHouse/ClickHouse/pull/64199) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64254](https://github.com/ClickHouse/ClickHouse/issues/64254): Ignore `text_log` config when using Keeper. [#64218](https://github.com/ClickHouse/ClickHouse/pull/64218) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#64690](https://github.com/ClickHouse/ClickHouse/issues/64690): Fix Query Tree size validation. Closes [#63701](https://github.com/ClickHouse/ClickHouse/issues/63701). [#64377](https://github.com/ClickHouse/ClickHouse/pull/64377) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64409](https://github.com/ClickHouse/ClickHouse/issues/64409): Fix `Logical error: Bad cast` for `Buffer` table with `PREWHERE`. Fixes [#64172](https://github.com/ClickHouse/ClickHouse/issues/64172). [#64388](https://github.com/ClickHouse/ClickHouse/pull/64388) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64727](https://github.com/ClickHouse/ClickHouse/issues/64727): Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#64623](https://github.com/ClickHouse/ClickHouse/issues/64623): Fix an error `Cannot find column` in distributed queries with constant CTE in the `GROUP BY` key. [#64519](https://github.com/ClickHouse/ClickHouse/pull/64519) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64680](https://github.com/ClickHouse/ClickHouse/issues/64680): Fix [#64612](https://github.com/ClickHouse/ClickHouse/issues/64612). Do not rewrite aggregation if `-If` combinator is already used. [#64638](https://github.com/ClickHouse/ClickHouse/pull/64638) ([Dmitry Novik](https://github.com/novikd)). +* Backported in [#64942](https://github.com/ClickHouse/ClickHouse/issues/64942): Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64871](https://github.com/ClickHouse/ClickHouse/issues/64871): Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). + +#### CI Fix or Improvement (changelog entry is not required) + +* Backported in [#63364](https://github.com/ClickHouse/ClickHouse/issues/63364): Implement cumulative A Sync status. [#61464](https://github.com/ClickHouse/ClickHouse/pull/61464) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63338](https://github.com/ClickHouse/ClickHouse/issues/63338): Use `/commit/` to have the URLs in [reports](https://play.clickhouse.com/play?user=play#c2VsZWN0IGRpc3RpbmN0IGNvbW1pdF91cmwgZnJvbSBjaGVja3Mgd2hlcmUgY2hlY2tfc3RhcnRfdGltZSA+PSBub3coKSAtIGludGVydmFsIDEgbW9udGggYW5kIHB1bGxfcmVxdWVzdF9udW1iZXI9NjA1MzI=) like https://github.com/ClickHouse/ClickHouse/commit/44f8bc5308b53797bec8cccc3bd29fab8a00235d and not like https://github.com/ClickHouse/ClickHouse/commits/44f8bc5308b53797bec8cccc3bd29fab8a00235d. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63376](https://github.com/ClickHouse/ClickHouse/issues/63376):. [#63366](https://github.com/ClickHouse/ClickHouse/pull/63366) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#63571](https://github.com/ClickHouse/ClickHouse/issues/63571):. [#63551](https://github.com/ClickHouse/ClickHouse/pull/63551) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Backported in [#63651](https://github.com/ClickHouse/ClickHouse/issues/63651): Fix 02362_part_log_merge_algorithm flaky test. [#63635](https://github.com/ClickHouse/ClickHouse/pull/63635) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Backported in [#63828](https://github.com/ClickHouse/ClickHouse/issues/63828): Fix test_odbc_interaction from aarch64 [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63787](https://github.com/ClickHouse/ClickHouse/pull/63787) ([alesapin](https://github.com/alesapin)). +* Backported in [#63897](https://github.com/ClickHouse/ClickHouse/issues/63897): Fix test `test_catboost_evaluate` for aarch64. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63789](https://github.com/ClickHouse/ClickHouse/pull/63789) ([alesapin](https://github.com/alesapin)). +* Backported in [#63889](https://github.com/ClickHouse/ClickHouse/issues/63889): Remove HDFS from disks config for one integration test for arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63832](https://github.com/ClickHouse/ClickHouse/pull/63832) ([alesapin](https://github.com/alesapin)). +* Backported in [#63881](https://github.com/ClickHouse/ClickHouse/issues/63881): Bump version for old image in test_short_strings_aggregation to make it work on arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63836](https://github.com/ClickHouse/ClickHouse/pull/63836) ([alesapin](https://github.com/alesapin)). +* Backported in [#63919](https://github.com/ClickHouse/ClickHouse/issues/63919): Disable test `test_non_default_compression/test.py::test_preconfigured_deflateqpl_codec` on arm. [#61457](https://github.com/ClickHouse/ClickHouse/issues/61457). [#63839](https://github.com/ClickHouse/ClickHouse/pull/63839) ([alesapin](https://github.com/alesapin)). +* Backported in [#63971](https://github.com/ClickHouse/ClickHouse/issues/63971): Fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#64049](https://github.com/ClickHouse/ClickHouse/issues/64049): Add `ClickHouseVersion.copy` method. Create a branch release in advance without spinning out the release to increase the stability. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64078](https://github.com/ClickHouse/ClickHouse/issues/64078): The mime type is not 100% reliable for Python and shell scripts without shebangs; add a check for file extension. [#64062](https://github.com/ClickHouse/ClickHouse/pull/64062) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#64161](https://github.com/ClickHouse/ClickHouse/issues/64161): Add retries in git submodule update. [#64125](https://github.com/ClickHouse/ClickHouse/pull/64125) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#64589](https://github.com/ClickHouse/ClickHouse/issues/64589): Disabled `enable_vertical_final` setting by default. This feature should not be used because it has a bug: [#64543](https://github.com/ClickHouse/ClickHouse/issues/64543). [#64544](https://github.com/ClickHouse/ClickHouse/pull/64544) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#64880](https://github.com/ClickHouse/ClickHouse/issues/64880): This PR fixes an error when a user in a specific situation can escalate their privileges on the default database without necessary grants. [#64769](https://github.com/ClickHouse/ClickHouse/pull/64769) ([pufit](https://github.com/pufit)). + +#### NO CL CATEGORY + +* Backported in [#63306](https://github.com/ClickHouse/ClickHouse/issues/63306):. [#63297](https://github.com/ClickHouse/ClickHouse/pull/63297) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#63710](https://github.com/ClickHouse/ClickHouse/issues/63710):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#64363](https://github.com/ClickHouse/ClickHouse/issues/64363) to 24.4: Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts"'. [#64905](https://github.com/ClickHouse/ClickHouse/pull/64905) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* group_by_use_nulls strikes back [#62922](https://github.com/ClickHouse/ClickHouse/pull/62922) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add `FROM` keyword to `TRUNCATE ALL TABLES` [#63241](https://github.com/ClickHouse/ClickHouse/pull/63241) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* More checks for concurrently deleted files and dirs in system.remote_data_paths [#63274](https://github.com/ClickHouse/ClickHouse/pull/63274) ([Alexander Gololobov](https://github.com/davenger)). +* Try fix segfault in `MergeTreeReadPoolBase::createTask` [#63323](https://github.com/ClickHouse/ClickHouse/pull/63323) ([Antonio Andelic](https://github.com/antonio2368)). +* Skip unaccessible table dirs in system.remote_data_paths [#63330](https://github.com/ClickHouse/ClickHouse/pull/63330) ([Alexander Gololobov](https://github.com/davenger)). +* Workaround for `oklch()` inside canvas bug for firefox [#63404](https://github.com/ClickHouse/ClickHouse/pull/63404) ([Sergei Trifonov](https://github.com/serxa)). +* Cancel S3 reads properly when parallel reads are used [#63687](https://github.com/ClickHouse/ClickHouse/pull/63687) ([Antonio Andelic](https://github.com/antonio2368)). +* Userspace page cache: don't collect stats if cache is unused [#63730](https://github.com/ClickHouse/ClickHouse/pull/63730) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix sanitizers [#64090](https://github.com/ClickHouse/ClickHouse/pull/64090) ([Azat Khuzhin](https://github.com/azat)). +* Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts [#64363](https://github.com/ClickHouse/ClickHouse/pull/64363) ([Kruglov Pavel](https://github.com/Avogar)). +* CI: Critical bugfix category in PR template [#64480](https://github.com/ClickHouse/ClickHouse/pull/64480) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f7d84cce4b1..2f96daf4887 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,4 +1,5 @@ v24.5.1.1763-stable 2024-06-01 +v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 v24.3.3.102-lts 2024-05-01 v24.3.2.23-lts 2024-04-03 From f77b6096c405aaf8862582bf87a897e925de7a6a Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 7 Jun 2024 12:14:21 +0000 Subject: [PATCH 47/54] time_virtual_col: st_mtime, not st_mtim, because of darwin --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 09f0bd60859..aaf84f6f82c 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1371,7 +1371,7 @@ Chunk StorageFileSource::generate() struct stat file_stat; file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); current_file_size = file_stat.st_size; - current_file_last_modified = Poco::Timestamp::fromEpochTime(file_stat.st_mtim.tv_sec); + current_file_last_modified = Poco::Timestamp::fromEpochTime(file_stat.st_mtime); if (getContext()->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) continue; From b04fb116a5cd8c6064804a1949b97d31ccc95a56 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 11:26:12 +0200 Subject: [PATCH 48/54] Compiler happy. Developer unhappy --- src/Storages/StorageGenerateRandom.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index ca9c6fb3226..8852e468c5e 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -706,7 +706,7 @@ Pipe StorageGenerateRandom::read( if (query_limit && num_streams * max_block_size > query_limit) { /// We want to avoid spawning more streams than necessary - num_streams = std::min(num_streams, ((query_limit + max_block_size - 1) / max_block_size)); + num_streams = std::min(num_streams, static_cast(((query_limit + max_block_size - 1) / max_block_size))); } Pipes pipes; pipes.reserve(num_streams); From f50a951e8e3cf80652ea525b4232833c846507f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 7 Jun 2024 16:49:07 +0200 Subject: [PATCH 49/54] Fix innocuous data race in detectLanguage --- contrib/cld2 | 2 +- tests/queries/0_stateless/03168_cld2_tsan.reference | 2 ++ tests/queries/0_stateless/03168_cld2_tsan.sql | 10 ++++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03168_cld2_tsan.reference create mode 100644 tests/queries/0_stateless/03168_cld2_tsan.sql diff --git a/contrib/cld2 b/contrib/cld2 index bc6d493a2f6..217ba8b8805 160000 --- a/contrib/cld2 +++ b/contrib/cld2 @@ -1 +1 @@ -Subproject commit bc6d493a2f64ed1fc1c4c4b4294a542a04e04217 +Subproject commit 217ba8b8805b41557faadaa47bb6e99f2242eea3 diff --git a/tests/queries/0_stateless/03168_cld2_tsan.reference b/tests/queries/0_stateless/03168_cld2_tsan.reference new file mode 100644 index 00000000000..6c3cafd4a6d --- /dev/null +++ b/tests/queries/0_stateless/03168_cld2_tsan.reference @@ -0,0 +1,2 @@ +{'ja':0.62,'fr':0.36} +{'ja':0.62,'fr':0.36} diff --git a/tests/queries/0_stateless/03168_cld2_tsan.sql b/tests/queries/0_stateless/03168_cld2_tsan.sql new file mode 100644 index 00000000000..701a781c472 --- /dev/null +++ b/tests/queries/0_stateless/03168_cld2_tsan.sql @@ -0,0 +1,10 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: depends on cld2 + +-- https://github.com/ClickHouse/ClickHouse/issues/64931 +SELECT detectLanguageMixed(materialize('二兎を追う者は一兎をも得ず二兎を追う者は一兎をも得ず A vaincre sans peril, on triomphe sans gloire.')) +GROUP BY + GROUPING SETS ( + ('a', toUInt256(1)), + (stringToH3(toFixedString(toFixedString('85283473ffffff', 14), 14)))) +SETTINGS allow_experimental_nlp_functions = 1; From 3238bcf90af9dfb7d5ce86b834744f15c11cec7f Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 8 Jun 2024 09:48:56 +0200 Subject: [PATCH 50/54] Review changes --- .../functions/other-functions.md | 115 ++++++++---------- 1 file changed, 50 insertions(+), 65 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 4639f9d0991..5e63d9824b4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -228,29 +228,24 @@ Query: ```sql DROP TABLE IF EXISTS test; CREATE TABLE test (n UInt8) ENGINE = Memory; --- Insert 3 blocks: -INSERT INTO test VALUES (1); -INSERT INTO test VALUES (1),(2); -INSERT INTO test VALUES (1),(2),(3); -SELECT blockSize(), n FROM (SELECT * FROM test); +INSERT INTO test +SELECT * FROM system.numbers LIMIT 5; + +SELECT blockSize() +FROM test; ``` Result: ```response - ┌─blockSize()─┬─n─┐ -1. │ 1 │ 1 │ - └─────────────┴───┘ - ┌─blockSize()─┬─n─┐ -2. │ 3 │ 1 │ -3. │ 3 │ 2 │ -4. │ 3 │ 3 │ - └─────────────┴───┘ - ┌─blockSize()─┬─n─┐ -5. │ 2 │ 1 │ -6. │ 2 │ 2 │ - └─────────────┴───┘ + ┌─blockSize()─┐ +1. │ 5 │ +2. │ 5 │ +3. │ 5 │ +4. │ 5 │ +5. │ 5 │ + └─────────────┘ ``` ## byteSize @@ -3723,11 +3718,7 @@ Result: ## lowCardinalityIndices -For each row in the current [block](../../development/architecture.md/#block-block), returns the index of the value in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. - -:::note -The first unique value encountered in the block is enumerated from 1. -::: +Returns the position of a value in the dictionary of a [LowCardinality](../data-types/lowcardinality.md) column. Positions start at 1. Since LowCardinality have per-part dictionaries, this function may return different positions for the same value in different parts. **Syntax** @@ -3741,7 +3732,7 @@ lowCardinalityIndices(col) **Returned value** -- returns the index of the value in the dictionary of unique values, for each row in the current block. [UInt64](../data-types/int-uint.md). +- The position of the value in the dictionary of the current part. [UInt64](../data-types/int-uint.md). **Example** @@ -3751,10 +3742,10 @@ Query: DROP TABLE IF EXISTS test; CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; --- insert two blocks of data: +-- create two parts: -INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); -INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); +INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df'); +INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef'); SELECT s, lowCardinalityIndices(s) FROM test; ``` @@ -3762,29 +3753,24 @@ SELECT s, lowCardinalityIndices(s) FROM test; Result: ```response - ┌─s───┬─lowCardinalityIndices(s)─┐ -1. │ one │ 1 │ -2. │ two │ 2 │ -3. │ one │ 1 │ -4. │ one │ 1 │ -5. │ two │ 2 │ - └─────┴──────────────────────────┘ - ┌─s─────┬─lowCardinalityIndices(s)─┐ - 6. │ three │ 1 │ - 7. │ two │ 2 │ - 8. │ one │ 3 │ - 9. │ two │ 2 │ -10. │ two │ 2 │ -11. │ three │ 1 │ - └───────┴──────────────────────────┘ + ┌─s──┬─lowCardinalityIndices(s)─┐ +1. │ ab │ 1 │ +2. │ cd │ 2 │ +3. │ ab │ 1 │ +4. │ ab │ 1 │ +5. │ df │ 3 │ + └────┴──────────────────────────┘ + ┌─s──┬─lowCardinalityIndices(s)─┐ + 6. │ ef │ 1 │ + 7. │ cd │ 2 │ + 8. │ ab │ 3 │ + 9. │ cd │ 2 │ +10. │ ef │ 1 │ + └────┴──────────────────────────┘ ``` ## lowCardinalityKeys -For each row in the current [block](../../development/architecture.md/#block-block), returns the keys (unique values) in the dictionary of unique values for columns of [LowCardinality](../data-types/lowcardinality.md) type. - -:::note -If the column size is less than the dictionary size, then values will be cut. If it is greater, then defaults will be added. -::: +Returns the dictionary values of a [LowCardinality](../data-types/lowcardinality.md) column. If the block is smaller or larger than the dictionary size, the result will be truncated or extended with default values. Since LowCardinality have per-part dictionaries, this function may return different dictionary values in different parts. **Syntax** @@ -3798,7 +3784,7 @@ lowCardinalityIndices(col) **Returned value** -- returns the keys of the dictionary, for each row in the current block. [UInt64](../data-types/int-uint.md). +- The dictionary keys. [UInt64](../data-types/int-uint.md). **Example** @@ -3808,10 +3794,10 @@ Query: DROP TABLE IF EXISTS test; CREATE TABLE test (s LowCardinality(String)) ENGINE = Memory; --- insert two blocks of data: +-- create two parts: -INSERT INTO test VALUES ('one'),('two'),('one'),('one'),('two'); -INSERT INTO test VALUES ('three'),('two'),('one'),('two'),('two'),('three'); +INSERT INTO test VALUES ('ab'), ('cd'), ('ab'), ('ab'), ('df'); +INSERT INTO test VALUES ('ef'), ('cd'), ('ab'), ('cd'), ('ef'); SELECT s, lowCardinalityKeys(s) FROM test; ``` @@ -3819,19 +3805,18 @@ SELECT s, lowCardinalityKeys(s) FROM test; Result: ```response - ┌─s───┬─lowCardinalityKeys(s)─┐ -1. │ one │ │ -2. │ two │ one │ -3. │ one │ two │ -4. │ one │ │ -5. │ two │ │ - └─────┴───────────────────────┘ - ┌─s─────┬─lowCardinalityKeys(s)─┐ - 6. │ three │ │ - 7. │ two │ three │ - 8. │ one │ two │ - 9. │ two │ one │ -10. │ two │ │ -11. │ three │ │ - └───────┴───────────────────────┘ + ┌─s──┬─lowCardinalityKeys(s)─┐ +1. │ ef │ │ +2. │ cd │ ef │ +3. │ ab │ cd │ +4. │ cd │ ab │ +5. │ ef │ │ + └────┴───────────────────────┘ + ┌─s──┬─lowCardinalityKeys(s)─┐ + 6. │ ab │ │ + 7. │ cd │ ab │ + 8. │ ab │ cd │ + 9. │ ab │ df │ +10. │ df │ │ + └────┴───────────────────────┘ ``` From cdd2957a31fae18e44c2230a73e0770c545f793c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 9 Jun 2024 19:09:33 +0000 Subject: [PATCH 51/54] Move MergeTree setting docs into MergeTree settings docs page --- .../mergetree-family/mergetree.md | 136 ++++------------ .../settings/merge-tree-settings.md | 154 ++++++++++++++---- 2 files changed, 151 insertions(+), 139 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 803b753fe0a..0e53d7525b4 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -6,35 +6,26 @@ sidebar_label: MergeTree # MergeTree -The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most commonly used and most robust ClickHouse table engines. +The `MergeTree` engine and other engines of the `MergeTree` family (e.g. `ReplacingMergeTree`, `AggregatingMergeTree` ) are the most commonly used and most robust table engines in ClickHouse. -Engines in the `MergeTree` family are designed for inserting a very large amount of data into a table. The data is quickly written to the table part by part, then rules are applied for merging the parts in the background. This method is much more efficient than continually rewriting the data in storage during insert. +`MergeTree`-family table engines are designed for high data ingest rates and huge data volumes. +Insert operations create table parts which are merged by a background process with other table parts. -Main features: +Main features of `MergeTree`-family table engines. -- Stores data sorted by primary key. +- The table's primary key determines the sort order within each table part (clustered index). The primary key also does not reference individual rows but blocks of 8192 rows called granules. This makes primary keys of huge data sets small enough to remain loaded in main memory, while still providing fast access to on-disk data. - This allows you to create a small sparse index that helps find data faster. +- Tables can be partitioned using an arbitrary partition expression. Partition pruning ensures partitions are omitted from reading when the query allows it. -- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified. +- Data can be replicated across multiple cluster nodes for high availability, failover, and zero downtime upgrades. See [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - ClickHouse supports certain operations with partitions that are more efficient than general operations on the same data with the same result. ClickHouse also automatically cuts off the partition data where the partitioning key is specified in the query. +- `MergeTree` table engines support various statistics kinds and sampling methods to help query optimization. -- Data replication support. - - The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - -- Data sampling support. - - If necessary, you can set the data sampling method in the table. - -:::info -The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family. +:::note +Despite a similar name, the [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine is different from `*MergeTree` engines. ::: -If you need to update rows frequently, we recommend using the [`ReplacingMergeTree`](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md) table engine. Using `ALTER TABLE my_table UPDATE` to update rows triggers a mutation, which causes parts to be re-written and uses IO/resources. With `ReplacingMergeTree`, you can simply insert the updated rows and the old rows will be replaced according to the table sorting key. - -## Creating a Table {#table_engine-mergetree-creating-a-table} +## Creating Tables {#table_engine-mergetree-creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -59,23 +50,24 @@ ORDER BY expr [SETTINGS name = value, ...] ``` -For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md). +For a detailed description of the parameters, see the [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md) statement ### Query Clauses {#mergetree-query-clauses} #### ENGINE -`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters. +`ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine has no parameters. #### ORDER_BY `ORDER BY` — The sorting key. -A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. +A tuple of column names or arbitrary expressions. Example: `ORDER BY (CounterID + 1, EventDate)`. -ClickHouse uses the sorting key as a primary key if the primary key is not defined explicitly by the `PRIMARY KEY` clause. +If no primary key is defined (i.e. `PRIMARY KEY` was not specified), ClickHouse uses the the sorting key as primary key. -Use the `ORDER BY tuple()` syntax, if you do not need sorting, or set `create_table_empty_primary_key_by_default` to `true` to use the `ORDER BY tuple()` syntax by default. See [Selecting the Primary Key](#selecting-the-primary-key). +If no sorting is required, you can use syntax `ORDER BY tuple()`. +Alternatively, if setting `create_table_empty_primary_key_by_default` is enabled, `ORDER BY tuple()` is implicitly added to `CREATE TABLE` statements. See [Selecting a Primary Key](#selecting-a-primary-key). #### PARTITION BY @@ -87,100 +79,32 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da `PRIMARY KEY` — The primary key if it [differs from the sorting key](#choosing-a-primary-key-that-differs-from-the-sorting-key). Optional. -By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. +Specifying a sorting key (using `ORDER BY` clause) implicitly specifies a primary key. +It is usually not necessary to specify the primary key in addition to the primary key. #### SAMPLE BY -`SAMPLE BY` — An expression for sampling. Optional. +`SAMPLE BY` — A sampling expression. Optional. -If a sampling expression is used, the primary key must contain it. The result of a sampling expression must be an unsigned integer. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. +If specified, it must be contained in the primary key. +The sampling expression must result in an unsigned integer. + +Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. #### TTL -`TTL` — A list of rules specifying storage duration of rows and defining logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. +`TTL` — A list of rules that specify the storage duration of rows and the logic of automatic parts movement [between disks and volumes](#table_engine-mergetree-multiple-volumes). Optional. -Expression must have one `Date` or `DateTime` column as a result. Example: -``` -TTL date + INTERVAL 1 DAY -``` +Expression must result in a `Date` or `DateTime`, e.g. `TTL date + INTERVAL 1 DAY`. Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an action to be done with the part if the expression is satisfied (reaches current time): removal of expired rows, moving a part (if expression is satisfied for all rows in a part) to specified disk (`TO DISK 'xxx'`) or to volume (`TO VOLUME 'xxx'`), or aggregating values in expired rows. Default type of the rule is removal (`DELETE`). List of multiple rules can be specified, but there should be no more than one `DELETE` rule. + For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) -### SETTINGS -Additional parameters that control the behavior of the `MergeTree` (optional): +#### Settings -#### index_granularity - -`index_granularity` — Maximum number of data rows between the marks of an index. Default value: 8192. See [Data Storage](#mergetree-data-storage). - -#### index_granularity_bytes - -`index_granularity_bytes` — Maximum size of data granules in bytes. Default value: 10Mb. To restrict the granule size only by number of rows, set to 0 (not recommended). See [Data Storage](#mergetree-data-storage). - -#### min_index_granularity_bytes - -`min_index_granularity_bytes` — Min allowed size of data granules in bytes. Default value: 1024b. To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. See [Data Storage](#mergetree-data-storage). - -#### enable_mixed_granularity_parts - -`enable_mixed_granularity_parts` — Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries. - -#### use_minimalistic_part_header_in_zookeeper - -`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”. - -#### min_merge_bytes_to_use_direct_io - -`min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes. - -#### merge_with_ttl_timeout - -`merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours). -#### merge_with_recompression_ttl_timeout - -`merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours). - -#### try_fetch_recompressed_part_timeout - -`try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours). - -#### write_final_mark - -`write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Don’t turn it off. - -#### merge_max_block_size - -`merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192. - -#### storage_policy - -`storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes). - -#### min_bytes_for_wide_part - -`min_bytes_for_wide_part`, `min_rows_for_wide_part` — Minimum number of bytes/rows in a data part that can be stored in `Wide` format. You can set one, both or none of these settings. See [Data Storage](#mergetree-data-storage). - -#### max_parts_in_total - -`max_parts_in_total` — Maximum number of parts in all partitions. - -#### max_compress_block_size - -`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - -#### min_compress_block_size - -`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. - -#### max_partitions_to_read - -`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. - -#### allow_experimental_optimized_row_order - -`allow_experimental_optimized_row_order` - Experimental. Enables the optimization of the row order during inserts to improve the compressability of the data for compression codecs (e.g. LZ4). Analyzes and reorders the data, and thus increases the CPU overhead of inserts. +See [MergeTree Settings](../../../operations/settings/merge-tree-settings.md). **Example of Sections Setting** @@ -270,7 +194,7 @@ ClickHouse does not require a unique primary key. You can insert multiple rows w You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses but it is strongly discouraged. To allow this feature, turn on the [allow_nullable_key](/docs/en/operations/settings/settings.md/#allow-nullable-key) setting. The [NULLS_LAST](/docs/en/sql-reference/statements/select/order-by.md/#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. -### Selecting the Primary Key {#selecting-the-primary-key} +### Selecting a Primary Key {#selecting-a-primary-key} The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may: diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index c3f303dcd38..d791683ac2b 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -3,9 +3,126 @@ slug: /en/operations/settings/merge-tree-settings title: "MergeTree tables settings" --- -The values of `merge_tree` settings (for all MergeTree tables) can be viewed in the table `system.merge_tree_settings`, they can be overridden in `config.xml` in the `merge_tree` section, or set in the `SETTINGS` section of each table. +System table `system.merge_tree_settings` shows the globally set MergeTree settings. -These are example overrides for `max_suspicious_broken_parts`: +MergeTree settings can be set in the `merge_tree` section of the server config file, or specified for each `MergeTree` table individually in +the `SETTINGS` clause of the `CREATE TABLE` statement. + +Example for customizing setting `max_suspicious_broken_parts`: + +Configure the default for all `MergeTree` tables in the server configuration file: + +``` text + + 5 + +``` + +Set for a particular table: + +``` sql +CREATE TABLE tab +( + `A` Int64 +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS max_suspicious_broken_parts = 500; +``` + +Change the settings for a particular table using `ALTER TABLE ... MODIFY SETTING`: + +```sql +ALTER TABLE tab MODIFY SETTING max_suspicious_broken_parts = 100; + +-- reset to global default (value from system.merge_tree_settings) +ALTER TABLE tab RESET SETTING max_suspicious_broken_parts; +``` + +## index_granularity + +Maximum number of data rows between the marks of an index. + +Default value: 8192. + +## index_granularity_bytes + +Maximum size of data granules in bytes. + +Default value: 10Mb. + +To restrict the granule size only by number of rows, set to 0 (not recommended). + +## min_index_granularity_bytes + +Min allowed size of data granules in bytes. + +Default value: 1024b. + +To provide a safeguard against accidentally creating tables with very low index_granularity_bytes. + +## enable_mixed_granularity_parts + +Enables or disables transitioning to control the granule size with the `index_granularity_bytes` setting. Before version 19.11, there was only the `index_granularity` setting for restricting granule size. The `index_granularity_bytes` setting improves ClickHouse performance when selecting data from tables with big rows (tens and hundreds of megabytes). If you have tables with big rows, you can enable this setting for the tables to improve the efficiency of `SELECT` queries. + +## use_minimalistic_part_header_in_zookeeper + +Storage method of the data parts headers in ZooKeeper. If enabled, ZooKeeper stores less data. For details, see [here](../server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper). + +## min_merge_bytes_to_use_direct_io + +The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. +When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. +If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). +If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. + +Default value: `10 * 1024 * 1024 * 1024` bytes. + +## merge_with_ttl_timeout + +Minimum delay in seconds before repeating a merge with delete TTL. + +Default value: `14400` seconds (4 hours). + +## merge_with_recompression_ttl_timeout + +Minimum delay in seconds before repeating a merge with recompression TTL. + +Default value: `14400` seconds (4 hours). + +## write_final_mark + +Enables or disables writing the final index mark at the end of data part (after the last byte). + +Default value: 1. + +Don’t change or bad things will happen. + +## storage_policy + +Storage policy. + +## min_bytes_for_wide_part + +Minimum number of bytes/rows in a data part that can be stored in `Wide` format. +You can set one, both or none of these settings. + +## max_compress_block_size + +Maximum size of blocks of uncompressed data before compressing for writing to a table. +You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). +The value specified when table is created overrides the global value for this setting. + +## min_compress_block_size + +Minimum size of blocks of uncompressed data required for compression when writing the next mark. +You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). +The value specified when table is created overrides the global value for this setting. + +## max_partitions_to_read + +Limits the maximum number of partitions that can be accessed in one query. +You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. ## max_suspicious_broken_parts @@ -17,37 +134,6 @@ Possible values: Default value: 100. -Override example in `config.xml`: - -``` text - - 5 - -``` - -An example to set in `SETTINGS` for a particular table: - -``` sql -CREATE TABLE foo -( - `A` Int64 -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS max_suspicious_broken_parts = 500; -``` - -An example of changing the settings for a specific table with the `ALTER TABLE ... MODIFY SETTING` command: - -``` sql -ALTER TABLE foo - MODIFY SETTING max_suspicious_broken_parts = 100; - --- reset to default (use value from system.merge_tree_settings) -ALTER TABLE foo - RESET SETTING max_suspicious_broken_parts; -``` - ## parts_to_throw_insert {#parts-to-throw-insert} If the number of active parts in a single partition exceeds the `parts_to_throw_insert` value, `INSERT` is interrupted with the `Too many parts (N). Merges are processing significantly slower than inserts` exception. @@ -301,6 +387,8 @@ Default value: 10800 ## try_fetch_recompressed_part_timeout +Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. + Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression. Possible values: From d59a17014461048ec215a45ce742f813c17f95c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 10 Jun 2024 07:05:36 +0000 Subject: [PATCH 52/54] Docs for MergeTree: Capitalized SETTINGS --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 0e53d7525b4..f0c4e1b0e34 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -102,7 +102,7 @@ Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an ac For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) -#### Settings +#### SETTINGS See [MergeTree Settings](../../../operations/settings/merge-tree-settings.md). From d80cba90fb467b40d098c028c163fa1deba07dd9 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 10 Jun 2024 11:22:06 +0200 Subject: [PATCH 53/54] CI: Remove fuzzer build from normal CI run (bugfix) --- tests/ci/ci_config.py | 35 ++++++++++++++++++++++++++++++----- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 775e1f98b34..412a55b8534 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -685,9 +685,6 @@ class CIConfig: return result def get_job_parents(self, check_name: str) -> List[str]: - if check_name in self.builds_report_config: - return self.builds_report_config[check_name].builds - res = [] check_name = normalize_string(check_name) for config in ( @@ -903,10 +900,38 @@ CI_CONFIG = CIConfig( ), CILabels.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), CILabels.CI_SET_NORMAL_BUILDS: LabelConfig( - run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK] + run_jobs=[ + JobNames.STYLE_CHECK, + JobNames.BUILD_CHECK, + Build.PACKAGE_RELEASE, + Build.PACKAGE_AARCH64, + Build.PACKAGE_ASAN, + Build.PACKAGE_UBSAN, + Build.PACKAGE_TSAN, + Build.PACKAGE_MSAN, + Build.PACKAGE_DEBUG, + Build.BINARY_RELEASE, + Build.PACKAGE_RELEASE_COVERAGE, + Build.FUZZERS, + ] ), CILabels.CI_SET_SPECIAL_BUILDS: LabelConfig( - run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK_SPECIAL] + run_jobs=[ + JobNames.STYLE_CHECK, + JobNames.BUILD_CHECK_SPECIAL, + Build.BINARY_TIDY, + Build.BINARY_DARWIN, + Build.BINARY_AARCH64, + Build.BINARY_AARCH64_V80COMPAT, + Build.BINARY_FREEBSD, + Build.BINARY_DARWIN_AARCH64, + Build.BINARY_PPC64LE, + Build.BINARY_RISCV64, + Build.BINARY_S390X, + Build.BINARY_LOONGARCH64, + Build.BINARY_AMD64_COMPAT, + Build.BINARY_AMD64_MUSL, + ] ), CILabels.CI_SET_NON_REQUIRED: LabelConfig( run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] From 132aa996a3f25fa98f3edc1ad92bbc22725d4c8d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 10 Jun 2024 13:49:39 +0200 Subject: [PATCH 54/54] Revert "Fix duplicating Delete events in blob_storage_log" --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 ++--- src/Interpreters/SystemLog.cpp | 7 +------ 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index afc13251f5b..ae719f5cde4 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -382,7 +382,6 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { std::vector current_chunk; String keys; - size_t first_position = current_position; for (; current_position < objects.size() && current_chunk.size() < chunk_size_limit; ++current_position) { Aws::S3::Model::ObjectIdentifier obj; @@ -408,9 +407,9 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e { const auto * outcome_error = outcome.IsSuccess() ? nullptr : &outcome.GetError(); auto time_now = std::chrono::system_clock::now(); - for (size_t i = first_position; i < current_position; ++i) + for (const auto & object : objects) blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, - uri.bucket, objects[i].remote_path, objects[i].local_path, objects[i].bytes_size, + uri.bucket, object.remote_path, object.local_path, object.bytes_size, outcome_error, time_now); } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3b25deeb59d..5e0ce2cb0de 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -504,10 +504,6 @@ void SystemLog::flushImpl(const std::vector & to_flush, Block block(std::move(log_element_columns)); MutableColumns columns = block.mutateColumns(); - - for (auto & column : columns) - column->reserve(to_flush.size()); - for (const auto & elem : to_flush) elem.appendToBlock(columns); @@ -536,8 +532,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to flush system log {} with {} entries up to offset {}", - table_id.getNameForLogs(), to_flush.size(), to_flush_end)); + tryLogCurrentException(__PRETTY_FUNCTION__); } queue->confirm(to_flush_end);