From 29b7bf64d43828c51a6c947f08e66697921ddc86 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 21 Feb 2024 20:04:12 +0100 Subject: [PATCH 01/60] Fix issues with endpoint and prefix --- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 92 +++++++++++++++---- .../test.py | 26 ++++++ 2 files changed, 101 insertions(+), 17 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 72c4abee5c9..ef24bdcf951 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -23,8 +23,32 @@ namespace ErrorCodes struct AzureBlobStorageEndpoint { const String storage_account_url; + const String account_name; const String container_name; + const String prefix; const std::optional container_already_exists; + + String getEndpoint() + { + String url = storage_account_url; + + if (!account_name.empty()) + url += "/" + account_name; + + url += "/" + container_name + "/" + prefix; + + return url; + } + + String getEndpointWithoutContainer() + { + String url = storage_account_url; + + if (!account_name.empty()) + url += "/" + account_name; + + return url; + } }; @@ -58,28 +82,64 @@ void validateContainerName(const String & container_name) AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { - std::string storage_url; - if (config.has(config_prefix + ".storage_account_url")) + String storage_url; + String account_name; + String container_name; + String prefix; + if (config.has(config_prefix + ".endpoint")) + { + String endpoint = config.getString(config_prefix + ".endpoint"); + + /// For authenitication methods other than managed identity (Eg: SAS, Workload Identity), + /// account name is not present in the endpoint + /// 'endpoint_contains_account_name' bool is used to understand how to split the endpoint (default : true) + + bool endpoint_contains_account_name = config.getBool(config_prefix + ".endpoint_contains_account_name", true); + + size_t pos = endpoint.find("//"); + assert (pos != std::string::npos); + + if (endpoint_contains_account_name) + { + size_t pos_begin = endpoint.find('/', pos+2); + size_t pos_end = endpoint.find('/',pos_begin+1); + account_name = endpoint.substr(pos_begin+1,(pos_end-pos_begin)-1); + + size_t cont_pos_end = endpoint.find('/', pos_end+1); + container_name = endpoint.substr(pos_end+1,(cont_pos_end-pos_end)-1); + prefix = endpoint.substr(cont_pos_end+1); + storage_url = endpoint.substr(0,pos_begin); + } + else + { + size_t pos_begin = endpoint.find('/', pos+2); + size_t pos_end = endpoint.find('/',pos_begin+1); + container_name = endpoint.substr(pos_begin+1,(pos_end-pos_begin)-1); + + container_name = endpoint.substr(pos_end+1); + storage_url = endpoint.substr(0,pos_begin); + } + } + else if (config.has(config_prefix + ".connection_string")) + { + storage_url = config.getString(config_prefix + ".connection_string"); + container_name = config.getString(config_prefix + ".container_name"); + } + else if (config.has(config_prefix + ".storage_account_url")) { storage_url = config.getString(config_prefix + ".storage_account_url"); validateStorageAccountUrl(storage_url); + container_name = config.getString(config_prefix + ".container_name"); } else - { - if (config.has(config_prefix + ".connection_string")) - storage_url = config.getString(config_prefix + ".connection_string"); - else if (config.has(config_prefix + ".endpoint")) - storage_url = config.getString(config_prefix + ".endpoint"); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected either `connection_string` or `endpoint` in config"); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected either `storage_account_url` or `connection_string` or `endpoint` in config"); - String container_name = config.getString(config_prefix + ".container_name", "default-container"); - validateContainerName(container_name); + if (!container_name.empty()) + validateContainerName(container_name); std::optional container_already_exists {}; if (config.has(config_prefix + ".container_already_exists")) container_already_exists = {config.getBool(config_prefix + ".container_already_exists")}; - return {storage_url, container_name, container_already_exists}; + return {storage_url, account_name, container_name, prefix, container_already_exists}; } @@ -133,15 +193,13 @@ std::unique_ptr getAzureBlobContainerClient( { auto endpoint = processAzureBlobStorageEndpoint(config, config_prefix); auto container_name = endpoint.container_name; - auto final_url = container_name.empty() - ? endpoint.storage_account_url - : (std::filesystem::path(endpoint.storage_account_url) / container_name).string(); + auto final_url = endpoint.getEndpoint(); if (endpoint.container_already_exists.value_or(false)) return getAzureBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); auto blob_service_client = getAzureBlobStorageClientWithAuth( - endpoint.storage_account_url, container_name, config, config_prefix); + endpoint.getEndpointWithoutContainer(), container_name, config, config_prefix); try { diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index f3e113c95d3..de8de347cdf 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -632,3 +632,29 @@ def test_endpoint(cluster): ) assert 10 == int(node.query("SELECT count() FROM test")) + +def test_endpoint_new_container(cluster): + node = cluster.instances[NODE_NAME] + account_name = "devstoreaccount1" + container_name = "cont3" + data_prefix = "data_prefix" + port = cluster.azurite_port + + node.query( + f""" + DROP TABLE IF EXISTS test SYNC; + + CREATE TABLE test (a Int32) + ENGINE = MergeTree() ORDER BY tuple() + SETTINGS disk = disk( + type = azure_blob_storage, + endpoint = 'http://azurite1:{port}/{account_name}/{container_name}/{data_prefix}', + account_name = 'devstoreaccount1', + account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', + skip_access_check = 0); + + INSERT INTO test SELECT number FROM numbers(10); + """ + ) + + assert 10 == int(node.query("SELECT count() FROM test")) From 3b11218e74a61dca93fa38f9f635a5cc8ddf3c1d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 21 Feb 2024 19:21:14 +0000 Subject: [PATCH 02/60] Automatic style fix --- tests/integration/test_merge_tree_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index de8de347cdf..2e72bc78524 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -633,6 +633,7 @@ def test_endpoint(cluster): assert 10 == int(node.query("SELECT count() FROM test")) + def test_endpoint_new_container(cluster): node = cluster.instances[NODE_NAME] account_name = "devstoreaccount1" From f1d5892d50c4d25a0aa3f14fcc4907655b27d030 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 22 Feb 2024 10:06:33 +0100 Subject: [PATCH 03/60] Fix style check --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index ef24bdcf951..659adb8e505 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -90,8 +90,7 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr { String endpoint = config.getString(config_prefix + ".endpoint"); - /// For authenitication methods other than managed identity (Eg: SAS, Workload Identity), - /// account name is not present in the endpoint + /// For some authentication methods account name is not present in the endpoint /// 'endpoint_contains_account_name' bool is used to understand how to split the endpoint (default : true) bool endpoint_contains_account_name = config.getBool(config_prefix + ".endpoint_contains_account_name", true); From 0eefab131dc120571eb5cd8b451d654b425fdc0f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 22 Feb 2024 11:05:33 +0100 Subject: [PATCH 04/60] Updated pos check --- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 67 ++++++++++++++----- .../test.py | 25 +++++++ 2 files changed, 77 insertions(+), 15 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 659adb8e505..9d11bb23578 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -35,7 +35,11 @@ struct AzureBlobStorageEndpoint if (!account_name.empty()) url += "/" + account_name; - url += "/" + container_name + "/" + prefix; + if (!container_name.empty()) + url += "/" + container_name; + + if (!prefix.empty()) + url += "/" + prefix; return url; } @@ -92,31 +96,64 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr /// For some authentication methods account name is not present in the endpoint /// 'endpoint_contains_account_name' bool is used to understand how to split the endpoint (default : true) - bool endpoint_contains_account_name = config.getBool(config_prefix + ".endpoint_contains_account_name", true); size_t pos = endpoint.find("//"); - assert (pos != std::string::npos); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected '//' in endpoint"); if (endpoint_contains_account_name) { - size_t pos_begin = endpoint.find('/', pos+2); - size_t pos_end = endpoint.find('/',pos_begin+1); - account_name = endpoint.substr(pos_begin+1,(pos_end-pos_begin)-1); + size_t acc_pos_begin = endpoint.find('/', pos+2); + if (acc_pos_begin == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected account_name in endpoint"); - size_t cont_pos_end = endpoint.find('/', pos_end+1); - container_name = endpoint.substr(pos_end+1,(cont_pos_end-pos_end)-1); - prefix = endpoint.substr(cont_pos_end+1); - storage_url = endpoint.substr(0,pos_begin); + storage_url = endpoint.substr(0,acc_pos_begin); + size_t acc_pos_end = endpoint.find('/',acc_pos_begin+1); + + if (acc_pos_end != std::string::npos) + { + account_name = endpoint.substr(acc_pos_begin+1,(acc_pos_end-acc_pos_begin)-1); + + size_t cont_pos_end = endpoint.find('/', acc_pos_end+1); + + if (cont_pos_end != std::string::npos) + { + container_name = endpoint.substr(acc_pos_end+1,(cont_pos_end-acc_pos_end)-1); + prefix = endpoint.substr(cont_pos_end+1); + } + else + { + container_name = endpoint.substr(acc_pos_end+1); + } + } + else + { + account_name = endpoint.substr(acc_pos_begin+1); + } } else { - size_t pos_begin = endpoint.find('/', pos+2); - size_t pos_end = endpoint.find('/',pos_begin+1); - container_name = endpoint.substr(pos_begin+1,(pos_end-pos_begin)-1); + size_t cont_pos_begin = endpoint.find('/', pos+2); + if (cont_pos_begin != std::string::npos) + { + storage_url = endpoint.substr(0,cont_pos_begin); + size_t cont_pos_end = endpoint.find('/',cont_pos_begin+1); - container_name = endpoint.substr(pos_end+1); - storage_url = endpoint.substr(0,pos_begin); + if (cont_pos_end != std::string::npos) + { + container_name = endpoint.substr(cont_pos_begin+1,(cont_pos_end-cont_pos_begin)-1); + prefix = endpoint.substr(cont_pos_end+1); + } + else + { + container_name = endpoint.substr(cont_pos_begin+1); + } + } + else + { + storage_url = endpoint; + } } } else if (config.has(config_prefix + ".connection_string")) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 2e72bc78524..eef5a230b40 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -659,3 +659,28 @@ def test_endpoint_new_container(cluster): ) assert 10 == int(node.query("SELECT count() FROM test")) + +def test_endpoint_without_prefix(cluster): + node = cluster.instances[NODE_NAME] + account_name = "devstoreaccount1" + container_name = "cont4" + port = cluster.azurite_port + + node.query( + f""" + DROP TABLE IF EXISTS test SYNC; + + CREATE TABLE test (a Int32) + ENGINE = MergeTree() ORDER BY tuple() + SETTINGS disk = disk( + type = azure_blob_storage, + endpoint = 'http://azurite1:{port}/{account_name}/{container_name}', + account_name = 'devstoreaccount1', + account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', + skip_access_check = 0); + + INSERT INTO test SELECT number FROM numbers(10); + """ + ) + + assert 10 == int(node.query("SELECT count() FROM test")) From 595ac95a5b25221a6a5ec8d72ea06a6b01777da6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Feb 2024 10:15:46 +0000 Subject: [PATCH 05/60] Automatic style fix --- tests/integration/test_merge_tree_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index eef5a230b40..0f66399ae0a 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -660,6 +660,7 @@ def test_endpoint_new_container(cluster): assert 10 == int(node.query("SELECT count() FROM test")) + def test_endpoint_without_prefix(cluster): node = cluster.instances[NODE_NAME] account_name = "devstoreaccount1" From a768f40adc3e656df036361aa132fe21e05f7dba Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Feb 2024 14:58:51 +0100 Subject: [PATCH 06/60] Add tailscale installation to the AMI preparation --- tests/ci/worker/prepare-ci-ami.sh | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index c27d956c834..5120f74c95a 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -138,6 +138,21 @@ dpkg -i /tmp/amazon-cloudwatch-agent.deb aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json systemctl enable amazon-cloudwatch-agent.service + +echo "Install tailscale" +# Build get-authkey for tailscale +docker run --rm -v /usr/local/bin/:/host-local-bin -i golang:alpine sh -ex <<'EOF' + CGO_ENABLED=0 go install -tags tag:svc-core-ci-github tailscale.com/cmd/get-authkey@main + mv /go/bin/get-authkey /host-local-bin +EOF + +# install tailscale +curl -fsSL "https://pkgs.tailscale.com/stable/ubuntu/$(lsb_release -cs).noarmor.gpg" > /usr/share/keyrings/tailscale-archive-keyring.gpg +curl -fsSL "https://pkgs.tailscale.com/stable/ubuntu/$(lsb_release -cs).tailscale-keyring.list" > /etc/apt/sources.list.d/tailscale.list +apt-get update +apt-get install tailscale --yes --no-install-recommends + + # The following line is used in aws TOE check. touch /var/tmp/clickhouse-ci-ami.success # END OF THE SCRIPT From be2ec93a3f3a649d62e214e83f8d17053facd356 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 20 Feb 2024 16:35:30 +0100 Subject: [PATCH 07/60] Remove commented code in ExpressionActions::execute() This information will be added to the exception anyway. Signed-off-by: Azat Khuzhin --- src/Interpreters/ExpressionActions.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 1bd1e2c318f..f86febb481d 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -736,10 +736,6 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) { executeAction(action, execution_context, dry_run); checkLimits(execution_context.columns); - - //std::cerr << "Action: " << action.toString() << std::endl; - //for (const auto & col : execution_context.columns) - // std::cerr << col.dumpStructure() << std::endl; } catch (Exception & e) { From 0cb6f205d7fa8a4b790879eae8d0ce4ae03cabc4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 20 Feb 2024 16:16:46 +0100 Subject: [PATCH 08/60] Fix actions execution during preliminary filtering (PK, partition pruning) Previously preliminary filtering works incorrectly if input columns had been used multiple times, because: - position initialized only for the first user - column had been moved, so the second user will get nullptr And this was the case when it had been required multiple times. Consider the following example: select * from data final prewhere indexHint(_partition_id = 'all') or indexHint(_partition_id = 'all') Actions for this PREWHERE is the following: (lldb) p actions.__ptr_->dumpActions() (std::string) $1 = "input: _partition_id LowCardinality(String) _partition_id LowCardinality(String) actions: INPUT : 0 -> _partition_id LowCardinality(String) : 0 COLUMN Const(String) -> 'all' String : 1 COLUMN Const(String) -> 'UInt8'_String String : 2 INPUT : 1 -> _partition_id LowCardinality(String) : 3 COLUMN Const(String) -> 'all' String : 4 COLUMN Const(String) -> 'UInt8'_String String : 5 FUNCTION equals(_partition_id :: 0, 'all' :: 1) -> equals(_partition_id, 'all') LowCardinality(UInt8) : 6 FUNCTION equals(_partition_id :: 3, 'all' :: 4) -> equals(_partition_id, 'all') LowCardinality(UInt8) : 1 FUNCTION _CAST(equals(_partition_id, 'all') :: 6, 'UInt8'_String :: 2) -> _CAST(equals(_partition_id, 'all'), 'UInt8'_String) UInt8 : 4 FUNCTION _CAST(equals(_partition_id, 'all') :: 1, 'UInt8'_String :: 5) -> _CAST(equals(_partition_id, 'all'), 'UInt8'_String) UInt8 : 2 FUNCTION or(_CAST(equals(_partition_id, 'all'), 'UInt8'_String) :: 4, _CAST(equals(_partition_id, 'all'), 'UInt8'_String) :: 2) -> or(indexHint(), indexHint())"... It has _partition_id column in input multiple times. So fix this by adding a flag (`allow_duplicates_in_input`), if set, columns will be fully initialized. v2: fix "Position out of bound in Block::erase()" Signed-off-by: Azat Khuzhin --- src/Interpreters/ExpressionActions.cpp | 26 ++++++++++++++----- src/Interpreters/ExpressionActions.h | 10 +++++-- src/Storages/VirtualColumnUtils.cpp | 6 ++--- ...inary_filters_duplicated_columns.reference | 2 ++ ...preliminary_filters_duplicated_columns.sql | 6 +++++ ..._columns_SimpleAggregateFunction.reference | 1 + ...icated_columns_SimpleAggregateFunction.sql | 5 ++++ 7 files changed, 44 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns.reference create mode 100644 tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns.sql create mode 100644 tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns_SimpleAggregateFunction.reference create mode 100644 tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns_SimpleAggregateFunction.sql diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index f86febb481d..383bbbae0dd 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -563,7 +563,7 @@ namespace }; } -static void executeAction(const ExpressionActions::Action & action, ExecutionContext & execution_context, bool dry_run) +static void executeAction(const ExpressionActions::Action & action, ExecutionContext & execution_context, bool dry_run, bool allow_duplicates_in_input) { auto & inputs = execution_context.inputs; auto & columns = execution_context.columns; @@ -694,14 +694,19 @@ static void executeAction(const ExpressionActions::Action & action, ExecutionCon action.node->result_name); } else - columns[action.result_position] = std::move(inputs[pos]); + { + if (allow_duplicates_in_input) + columns[action.result_position] = inputs[pos]; + else + columns[action.result_position] = std::move(inputs[pos]); + } break; } } } -void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) const +void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run, bool allow_duplicates_in_input) const { ExecutionContext execution_context { @@ -722,7 +727,8 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) if (execution_context.inputs_pos[input_pos] < 0) { execution_context.inputs_pos[input_pos] = pos; - break; + if (!allow_duplicates_in_input) + break; } } } @@ -734,7 +740,7 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) { try { - executeAction(action, execution_context, dry_run); + executeAction(action, execution_context, dry_run, allow_duplicates_in_input); checkLimits(execution_context.columns); } catch (Exception & e) @@ -748,6 +754,12 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) { block.clear(); } + else if (allow_duplicates_in_input) + { + /// This case is the same as when the input is projected + /// since we do not need any input columns. + block.clear(); + } else { ::sort(execution_context.inputs_pos.rbegin(), execution_context.inputs_pos.rend()); @@ -770,11 +782,11 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) num_rows = execution_context.num_rows; } -void ExpressionActions::execute(Block & block, bool dry_run) const +void ExpressionActions::execute(Block & block, bool dry_run, bool allow_duplicates_in_input) const { size_t num_rows = block.rows(); - execute(block, num_rows, dry_run); + execute(block, num_rows, dry_run, allow_duplicates_in_input); if (!block) block.insert({DataTypeUInt8().createColumnConst(num_rows, 0), std::make_shared(), "_dummy"}); diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index db6670c50b9..cb467004d29 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -98,9 +98,15 @@ public: const NamesAndTypesList & getRequiredColumnsWithTypes() const { return required_columns; } /// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns. - void execute(Block & block, size_t & num_rows, bool dry_run = false) const; + /// + /// @param allow_duplicates_in_input - actions are allowed to have + /// duplicated input (that will refer into the block). This is needed for + /// preliminary query filtering (filterBlockWithDAG()), because they just + /// pass available virtual columns, which cannot be moved in case they are + /// used multiple times. + void execute(Block & block, size_t & num_rows, bool dry_run = false, bool allow_duplicates_in_input = false) const; /// The same, but without `num_rows`. If result block is empty, adds `_dummy` column to keep block size. - void execute(Block & block, bool dry_run = false) const; + void execute(Block & block, bool dry_run = false, bool allow_duplicates_in_input = false) const; bool hasArrayJoin() const; void assertDeterministic() const; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 33ff6e7104f..07ac61c110d 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -223,7 +223,7 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block auto expression_actions = std::make_shared(actions); auto block_with_constants = block; - expression_actions->execute(block_with_constants); + expression_actions->execute(block_with_constants, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column); }; @@ -266,7 +266,7 @@ void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) auto actions = std::make_shared(dag); makeSets(actions, context); Block block_with_filter = block; - actions->execute(block_with_filter); + actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); /// Filter the block. String filter_column_name = dag->getOutputs().at(0)->result_name; @@ -313,7 +313,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex makeSets(actions, context); Block block_with_filter = block; - actions->execute(block_with_filter); + actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); /// Filter the block. String filter_column_name = expression_ast->getColumnName(); diff --git a/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns.reference b/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns.sql b/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns.sql new file mode 100644 index 00000000000..060f16f8945 --- /dev/null +++ b/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns.sql @@ -0,0 +1,6 @@ +-- It is special because actions cannot be reused for SimpleAggregateFunction (see https://github.com/ClickHouse/ClickHouse/pull/54436) +drop table if exists data; +create table data (key Int) engine=AggregatingMergeTree() order by tuple(); +insert into data values (0); +select * from data final prewhere indexHint(_partition_id = 'all') or indexHint(_partition_id = 'all'); +select * from data final prewhere indexHint(_partition_id = 'all') or indexHint(_partition_id = 'all') or indexHint(_partition_id = 'all'); diff --git a/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns_SimpleAggregateFunction.reference b/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns_SimpleAggregateFunction.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns_SimpleAggregateFunction.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns_SimpleAggregateFunction.sql b/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns_SimpleAggregateFunction.sql new file mode 100644 index 00000000000..97df883fa48 --- /dev/null +++ b/tests/queries/0_stateless/02995_preliminary_filters_duplicated_columns_SimpleAggregateFunction.sql @@ -0,0 +1,5 @@ +-- It is special because actions cannot be reused for SimpleAggregateFunction (see https://github.com/ClickHouse/ClickHouse/pull/54436) +drop table if exists data; +create table data (key SimpleAggregateFunction(max, Int)) engine=AggregatingMergeTree() order by tuple(); +insert into data values (0); +select * from data final prewhere indexHint(_partition_id = 'all') and key >= -1 where key >= 0; From 86694596b3f570520deeab591f768c33f700a67d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 23 Feb 2024 11:12:42 +0100 Subject: [PATCH 09/60] Fix flaky test --- .../test_merge_tree_azure_blob_storage/test.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 0f66399ae0a..2b533c7104e 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -613,7 +613,8 @@ def test_endpoint(cluster): container_client = cluster.blob_service_client.get_container_client(container_name) container_client.create_container() - node.query( + azure_query( + node, f""" DROP TABLE IF EXISTS test SYNC; @@ -641,7 +642,8 @@ def test_endpoint_new_container(cluster): data_prefix = "data_prefix" port = cluster.azurite_port - node.query( + azure_query( + node, f""" DROP TABLE IF EXISTS test SYNC; @@ -667,7 +669,8 @@ def test_endpoint_without_prefix(cluster): container_name = "cont4" port = cluster.azurite_port - node.query( + azure_query( + node, f""" DROP TABLE IF EXISTS test SYNC; From 5f39784093787fb5e0aaf957a403b104beb07d72 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Feb 2024 10:24:00 +0000 Subject: [PATCH 10/60] Automatic style fix --- .../integration/test_merge_tree_azure_blob_storage/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 2b533c7104e..b0373ffb811 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -629,7 +629,7 @@ def test_endpoint(cluster): skip_access_check = 0); INSERT INTO test SELECT number FROM numbers(10); - """ + """, ) assert 10 == int(node.query("SELECT count() FROM test")) @@ -657,7 +657,7 @@ def test_endpoint_new_container(cluster): skip_access_check = 0); INSERT INTO test SELECT number FROM numbers(10); - """ + """, ) assert 10 == int(node.query("SELECT count() FROM test")) @@ -684,7 +684,7 @@ def test_endpoint_without_prefix(cluster): skip_access_check = 0); INSERT INTO test SELECT number FROM numbers(10); - """ + """, ) assert 10 == int(node.query("SELECT count() FROM test")) From 758acd4268ab2840b138fea3b912e84d43f5e1ef Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Feb 2024 15:21:41 +0100 Subject: [PATCH 11/60] Setup tailscale on instances, create a common network script --- tests/ci/worker/dockerhub_proxy_template.sh | 14 +---------- tests/ci/worker/init_runner.sh | 14 +---------- tests/ci/worker/prepare-ci-ami.sh | 28 +++++++++++++++++++++ 3 files changed, 30 insertions(+), 26 deletions(-) diff --git a/tests/ci/worker/dockerhub_proxy_template.sh b/tests/ci/worker/dockerhub_proxy_template.sh index 7ca8d581df5..0e375dd5f04 100644 --- a/tests/ci/worker/dockerhub_proxy_template.sh +++ b/tests/ci/worker/dockerhub_proxy_template.sh @@ -1,19 +1,7 @@ #!/usr/bin/env bash set -xeuo pipefail -# Add cloudflare DNS as a fallback -# Get default gateway interface -IFACE=$(ip --json route list | jq '.[]|select(.dst == "default").dev' --raw-output) -# `Link 2 (eth0): 172.31.0.2` -ETH_DNS=$(resolvectl dns "$IFACE") || : -CLOUDFLARE_NS=1.1.1.1 -if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFLARE_NS"* ]]; then - # Cut the leading legend - ETH_DNS=${ETH_DNS#*: } - # shellcheck disable=SC2206 - new_dns=(${ETH_DNS} "$CLOUDFLARE_NS") - resolvectl dns "$IFACE" "${new_dns[@]}" -fi +bash /usr/local/share/scripts/init-network.sh # tune sysctl for network performance cat > /etc/sysctl.d/10-network-memory.conf << EOF diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 017d847739f..b211128cf10 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -60,19 +60,7 @@ export RUNNER_URL="https://github.com/${RUNNER_ORG}" INSTANCE_ID=$(ec2metadata --instance-id) export INSTANCE_ID -# Add cloudflare DNS as a fallback -# Get default gateway interface -IFACE=$(ip --json route list | jq '.[]|select(.dst == "default").dev' --raw-output) -# `Link 2 (eth0): 172.31.0.2` -ETH_DNS=$(resolvectl dns "$IFACE") || : -CLOUDFLARE_NS=1.1.1.1 -if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFLARE_NS"* ]]; then - # Cut the leading legend - ETH_DNS=${ETH_DNS#*: } - # shellcheck disable=SC2206 - new_dns=(${ETH_DNS} "$CLOUDFLARE_NS") - resolvectl dns "$IFACE" "${new_dns[@]}" -fi +bash /usr/local/share/scripts/init-network.sh # combine labels RUNNER_TYPE=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:runner-type'].Value" --output text) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 5120f74c95a..bd2b8e730b3 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -153,6 +153,34 @@ apt-get update apt-get install tailscale --yes --no-install-recommends +# Create a common script for the instances +mkdir /usr/local/share/scripts -p +cat > /usr/local/share/scripts/init-network.sh << 'EOF' +#!/usr/bin/env bash + +# Add cloudflare DNS as a fallback +# Get default gateway interface +IFACE=$(ip --json route list | jq '.[]|select(.dst == "default").dev' --raw-output) +# `Link 2 (eth0): 172.31.0.2` +ETH_DNS=$(resolvectl dns "$IFACE") || : +CLOUDFLARE_NS=1.1.1.1 +if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFLARE_NS"* ]]; then + # Cut the leading legend + ETH_DNS=${ETH_DNS#*: } + # shellcheck disable=SC2206 + new_dns=(${ETH_DNS} "$CLOUDFLARE_NS") + resolvectl dns "$IFACE" "${new_dns[@]}" +fi + +# Setup tailscale, the very first action +TS_API_CLIENT_ID=$(aws ssm get-parameter --region us-east-1 --name /tailscale/api-client-id --query 'Parameter.Value' --output text --with-decryption) +TS_API_CLIENT_SECRET=$(aws ssm get-parameter --region us-east-1 --name /tailscale/api-client-secret --query 'Parameter.Value' --output text --with-decryption) +export TS_API_CLIENT_ID TS_API_CLIENT_SECRET +TS_AUTHKEY=$(get-authkey -tags tag:svc-core-ci-github -reusable -ephemeral) +tailscale up --ssh --auth-key="$TS_AUTHKEY" --hostname="ci-runner-$INSTANCE_ID" +EOF + + # The following line is used in aws TOE check. touch /var/tmp/clickhouse-ci-ami.success # END OF THE SCRIPT From ff353823b6ac3e58a57e834f526f6e903b3d1708 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Feb 2024 16:18:22 +0100 Subject: [PATCH 12/60] Update the GH actions runner version --- tests/ci/worker/prepare-ci-ami.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index bd2b8e730b3..281dff5b1c2 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -9,7 +9,7 @@ set -xeuo pipefail echo "Running prepare script" export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.311.0 +export RUNNER_VERSION=2.313.0 export RUNNER_HOME=/home/ubuntu/actions-runner deb_arch() { From 6ec23a08791c02c6b381cdd4cdd02981ddb7205a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 26 Feb 2024 14:36:06 +0100 Subject: [PATCH 13/60] Updated docs for endpoint --- docs/en/engines/table-engines/integrations/azureBlobStorage.md | 2 ++ docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index c6525121667..70d696f9684 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -19,6 +19,8 @@ CREATE TABLE azure_blob_storage_table (name String, value UInt32) ### Engine parameters +- `endpoint` — AzureBlobStorage endpoint URL with container & prefix. Optionally can contain account_name if the authentication method used needs it. (http://azurite1:{port}/[account_name]{container_name}/{data_prefix}) or these parameters can be provided separately using storage_account_url, account_name & container. For specifying prefix, endpoint should be used. +- `endpoint_contains_account_name` - This flag is used to specify if endpoint contains account_name as it is only needed for certain authentication methods. - `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) - `container_name` - Container name - `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index f185c11bab3..cbceeb05c2a 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1236,6 +1236,8 @@ Configuration markup: ``` Connection parameters: +* `endpoint` — AzureBlobStorage endpoint URL with container & prefix. Optionally can contain account_name if the authentication method used needs it. (http://azurite1:{port}/[account_name]{container_name}/{data_prefix}) or these parameters can be provided separately using storage_account_url, account_name & container. For specifying prefix, endpoint should be used. +* `endpoint_contains_account_name` - This flag is used to specify if endpoint contains account_name as it is only needed for certain authentication methods. * `storage_account_url` - **Required**, Azure Blob Storage account URL, like `http://account.blob.core.windows.net` or `http://azurite1:10000/devstoreaccount1`. * `container_name` - Target container name, defaults to `default-container`. * `container_already_exists` - If set to `false`, a new container `container_name` is created in the storage account, if set to `true`, disk connects to the container directly, and if left unset, disk connects to the account, checks if the container `container_name` exists, and creates it if it doesn't exist yet. From ae309e6ea9972f801e8b80613ff2b2c6f3c67231 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 27 Feb 2024 11:12:19 +0100 Subject: [PATCH 14/60] Updated to set proper prefix for azure blob storage disk --- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 35 ------------------ .../AzureBlobStorage/AzureBlobStorageAuth.h | 37 +++++++++++++++++++ .../ObjectStorages/ObjectStorageFactory.cpp | 4 +- 3 files changed, 39 insertions(+), 37 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 9d11bb23578..5657eab1872 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -20,41 +20,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -struct AzureBlobStorageEndpoint -{ - const String storage_account_url; - const String account_name; - const String container_name; - const String prefix; - const std::optional container_already_exists; - - String getEndpoint() - { - String url = storage_account_url; - - if (!account_name.empty()) - url += "/" + account_name; - - if (!container_name.empty()) - url += "/" + container_name; - - if (!prefix.empty()) - url += "/" + prefix; - - return url; - } - - String getEndpointWithoutContainer() - { - String url = storage_account_url; - - if (!account_name.empty()) - url += "/" + account_name; - - return url; - } -}; - void validateStorageAccountUrl(const String & storage_account_url) { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h index 18e8bf159d5..20bf05d5ba6 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h @@ -10,9 +10,46 @@ namespace DB { +struct AzureBlobStorageEndpoint +{ + const String storage_account_url; + const String account_name; + const String container_name; + const String prefix; + const std::optional container_already_exists; + + String getEndpoint() + { + String url = storage_account_url; + + if (!account_name.empty()) + url += "/" + account_name; + + if (!container_name.empty()) + url += "/" + container_name; + + if (!prefix.empty()) + url += "/" + prefix; + + return url; + } + + String getEndpointWithoutContainer() + { + String url = storage_account_url; + + if (!account_name.empty()) + url += "/" + account_name; + + return url; + } +}; + std::unique_ptr getAzureBlobContainerClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix); +AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix); + std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/); } diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index b3626135177..d13cf1156f8 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -213,12 +213,12 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) const ContextPtr & context, bool /* skip_access_check */) -> ObjectStoragePtr { - String container_name = config.getString(config_prefix + ".container_name", "default-container"); + AzureBlobStorageEndpoint endpoint = processAzureBlobStorageEndpoint(config, config_prefix); return std::make_unique( name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), - container_name); + endpoint.prefix ? endpoint.container_name + "/" + endpoint.prefix : endpoint.container); }); } From 414c8da128d2f3cf6611f9642401a809193c22a7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 27 Feb 2024 18:09:21 +0100 Subject: [PATCH 15/60] Fix build --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index d13cf1156f8..9a3ae432eca 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -218,7 +218,7 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), - endpoint.prefix ? endpoint.container_name + "/" + endpoint.prefix : endpoint.container); + endpoint.prefix.empty() ? endpoint.container : endpoint.container_name + "/" + endpoint.prefix); }); } From ca05557659de3f7fcd14d47acf1d69a5da18d924 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 27 Feb 2024 19:16:36 +0100 Subject: [PATCH 16/60] Fix typo --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 9a3ae432eca..f4df579de73 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -218,7 +218,7 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), - endpoint.prefix.empty() ? endpoint.container : endpoint.container_name + "/" + endpoint.prefix); + endpoint.prefix.empty() ? endpoint.container_name : endpoint.container_name + "/" + endpoint.prefix); }); } From a46d7c91910d5077024f113377f42e469b48849e Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 28 Feb 2024 11:25:35 +0100 Subject: [PATCH 17/60] Updated docs and addressed comments --- .../mergetree-family/mergetree.md | 6 +- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 55 ++++++++---------- .../AzureBlobStorage/AzureObjectStorage.cpp | 6 +- .../AzureBlobStorage/AzureObjectStorage.h | 4 +- .../test.py | 58 +++++++++++++++++++ 5 files changed, 90 insertions(+), 39 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index cbceeb05c2a..5f66c2efecd 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1236,9 +1236,9 @@ Configuration markup: ``` Connection parameters: -* `endpoint` — AzureBlobStorage endpoint URL with container & prefix. Optionally can contain account_name if the authentication method used needs it. (http://azurite1:{port}/[account_name]{container_name}/{data_prefix}) or these parameters can be provided separately using storage_account_url, account_name & container. For specifying prefix, endpoint should be used. -* `endpoint_contains_account_name` - This flag is used to specify if endpoint contains account_name as it is only needed for certain authentication methods. -* `storage_account_url` - **Required**, Azure Blob Storage account URL, like `http://account.blob.core.windows.net` or `http://azurite1:10000/devstoreaccount1`. +* `endpoint` — AzureBlobStorage endpoint URL with container & prefix. Optionally can contain account_name if the authentication method used needs it. (`http://account.blob.core.windows.net:{port}/[account_name]{container_name}/{data_prefix}`) or these parameters can be provided separately using storage_account_url, account_name & container. For specifying prefix, endpoint should be used. +* `endpoint_contains_account_name` - This flag is used to specify if endpoint contains account_name as it is only needed for certain authentication methods. (Default : false) +* `storage_account_url` - Required if endpoint is not specified, Azure Blob Storage account URL, like `http://account.blob.core.windows.net` or `http://azurite1:10000/devstoreaccount1`. * `container_name` - Target container name, defaults to `default-container`. * `container_already_exists` - If set to `false`, a new container `container_name` is created in the storage account, if set to `true`, disk connects to the container directly, and if left unset, disk connects to the account, checks if the container `container_name` exists, and creates it if it doesn't exist yet. diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 5657eab1872..7dfee3bc760 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -60,8 +60,8 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr String endpoint = config.getString(config_prefix + ".endpoint"); /// For some authentication methods account name is not present in the endpoint - /// 'endpoint_contains_account_name' bool is used to understand how to split the endpoint (default : true) - bool endpoint_contains_account_name = config.getBool(config_prefix + ".endpoint_contains_account_name", true); + /// 'endpoint_contains_account_name' bool is used to understand how to split the endpoint (default : false) + bool endpoint_contains_account_name = config.getBool(config_prefix + ".endpoint_contains_account_name", false); size_t pos = endpoint.find("//"); if (pos == std::string::npos) @@ -76,48 +76,41 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr storage_url = endpoint.substr(0,acc_pos_begin); size_t acc_pos_end = endpoint.find('/',acc_pos_begin+1); - if (acc_pos_end != std::string::npos) + if (acc_pos_end == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected container_name in endpoint"); + + account_name = endpoint.substr(acc_pos_begin+1,(acc_pos_end-acc_pos_begin)-1); + + size_t cont_pos_end = endpoint.find('/', acc_pos_end+1); + + if (cont_pos_end != std::string::npos) { - account_name = endpoint.substr(acc_pos_begin+1,(acc_pos_end-acc_pos_begin)-1); - - size_t cont_pos_end = endpoint.find('/', acc_pos_end+1); - - if (cont_pos_end != std::string::npos) - { - container_name = endpoint.substr(acc_pos_end+1,(cont_pos_end-acc_pos_end)-1); - prefix = endpoint.substr(cont_pos_end+1); - } - else - { - container_name = endpoint.substr(acc_pos_end+1); - } + container_name = endpoint.substr(acc_pos_end+1,(cont_pos_end-acc_pos_end)-1); + prefix = endpoint.substr(cont_pos_end+1); } else { - account_name = endpoint.substr(acc_pos_begin+1); + container_name = endpoint.substr(acc_pos_end+1); } } else { size_t cont_pos_begin = endpoint.find('/', pos+2); - if (cont_pos_begin != std::string::npos) - { - storage_url = endpoint.substr(0,cont_pos_begin); - size_t cont_pos_end = endpoint.find('/',cont_pos_begin+1); - if (cont_pos_end != std::string::npos) - { - container_name = endpoint.substr(cont_pos_begin+1,(cont_pos_end-cont_pos_begin)-1); - prefix = endpoint.substr(cont_pos_end+1); - } - else - { - container_name = endpoint.substr(cont_pos_begin+1); - } + if (cont_pos_begin == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected container_name in endpoint"); + + storage_url = endpoint.substr(0,cont_pos_begin); + size_t cont_pos_end = endpoint.find('/',cont_pos_begin+1); + + if (cont_pos_end != std::string::npos) + { + container_name = endpoint.substr(cont_pos_begin+1,(cont_pos_end-cont_pos_begin)-1); + prefix = endpoint.substr(cont_pos_end+1); } else { - storage_url = endpoint; + container_name = endpoint.substr(cont_pos_begin+1); } } } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 74389aedb64..fba61c7c392 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -93,11 +93,11 @@ AzureObjectStorage::AzureObjectStorage( const String & name_, AzureClientPtr && client_, SettingsPtr && settings_, - const String & container_) + const String & object_namespace_) : name(name_) , client(std::move(client_)) , settings(std::move(settings_)) - , container(container_) + , object_namespace(object_namespace_) , log(getLogger("AzureObjectStorage")) { } @@ -379,7 +379,7 @@ std::unique_ptr AzureObjectStorage::cloneObjectStorage(const std name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), - container + object_namespace ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f16c35fb52c..8109b8eaf54 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -130,7 +130,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - String getObjectsNamespace() const override { return container ; } + String getObjectsNamespace() const override { return object_namespace ; } std::unique_ptr cloneObjectStorage( const std::string & new_namespace, @@ -154,7 +154,7 @@ private: /// client used to access the files in the Blob Storage cloud MultiVersion client; MultiVersion settings; - const String container; + const String object_namespace; /// container + prefix LoggerPtr log; }; diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index b0373ffb811..39bdf51eae3 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -623,6 +623,7 @@ def test_endpoint(cluster): SETTINGS disk = disk( type = azure_blob_storage, endpoint = 'http://azurite1:{port}/{account_name}/{container_name}/{data_prefix}', + endpoint_contains_account_name = 'true', account_name = 'devstoreaccount1', account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', container_already_exists = 1, @@ -652,6 +653,7 @@ def test_endpoint_new_container(cluster): SETTINGS disk = disk( type = azure_blob_storage, endpoint = 'http://azurite1:{port}/{account_name}/{container_name}/{data_prefix}', + endpoint_contains_account_name = 'true', account_name = 'devstoreaccount1', account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', skip_access_check = 0); @@ -679,6 +681,7 @@ def test_endpoint_without_prefix(cluster): SETTINGS disk = disk( type = azure_blob_storage, endpoint = 'http://azurite1:{port}/{account_name}/{container_name}', + endpoint_contains_account_name = 'true', account_name = 'devstoreaccount1', account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', skip_access_check = 0); @@ -688,3 +691,58 @@ def test_endpoint_without_prefix(cluster): ) assert 10 == int(node.query("SELECT count() FROM test")) + +def test_endpoint_error_check(cluster): + node = cluster.instances[NODE_NAME] + account_name = "devstoreaccount1" + port = cluster.azurite_port + + query = f""" + DROP TABLE IF EXISTS test SYNC; + + CREATE TABLE test (a Int32) + ENGINE = MergeTree() ORDER BY tuple() + SETTINGS disk = disk( + type = azure_blob_storage, + endpoint = 'http://azurite1:{port}/{account_name}', + endpoint_contains_account_name = 'true', + account_name = 'devstoreaccount1', + account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', + skip_access_check = 0); + """ + + expected_err_msg = "Expected container_name in endpoint" + assert expected_err_msg in azure_query(node, query, expect_error="true") + + query = f""" + DROP TABLE IF EXISTS test SYNC; + + CREATE TABLE test (a Int32) + ENGINE = MergeTree() ORDER BY tuple() + SETTINGS disk = disk( + type = azure_blob_storage, + endpoint = 'http://azurite1:{port}', + endpoint_contains_account_name = 'true', + account_name = 'devstoreaccount1', + account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', + skip_access_check = 0); + """ + + expected_err_msg = "Expected account_name in endpoint" + assert expected_err_msg in azure_query(node, query, expect_error="true") + + query = f""" + DROP TABLE IF EXISTS test SYNC; + + CREATE TABLE test (a Int32) + ENGINE = MergeTree() ORDER BY tuple() + SETTINGS disk = disk( + type = azure_blob_storage, + endpoint = 'http://azurite1:{port}', + account_name = 'devstoreaccount1', + account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', + skip_access_check = 0); + """ + + expected_err_msg = "Expected container_name in endpoint" + assert expected_err_msg in azure_query(node, query, expect_error="true") From 8145cd15b11e2873b9ede75d0e29df822b70f871 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 28 Feb 2024 10:34:40 +0000 Subject: [PATCH 18/60] Automatic style fix --- tests/integration/test_merge_tree_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 39bdf51eae3..27aa6c12610 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -692,6 +692,7 @@ def test_endpoint_without_prefix(cluster): assert 10 == int(node.query("SELECT count() FROM test")) + def test_endpoint_error_check(cluster): node = cluster.instances[NODE_NAME] account_name = "devstoreaccount1" From ae06e34ea9d23a06b7b90e9568a777c5d79ba859 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Wed, 28 Feb 2024 19:31:58 +0800 Subject: [PATCH 19/60] Auto-vectorize DB::andFilters DB::MergeTreeRangeReader::ReadResult::optimize adds the current filter to the final one either via DB::combineFilters when both filters have different sizes or otherwise via DB::andFilters, which performs the logical AND operation (&&) to two UInt8 columns. To vectorize DB::andFilters automatically, this commit performs NOT operation to the operands twice so that the non-zeros are converted to bool value of true (0x01) and the zeros to false (0x00), and via this change the logical AND (&&) could be safely replaced with the bitwise AND (&), which could be recognized and optimized by the compiler. --- .../MergeTree/MergeTreeRangeReader.cpp | 15 ++--- .../MergeTree/tests/gtest_combine_filters.cpp | 66 +++++++++++++++++++ 2 files changed, 71 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index cce7e56dda9..50d1216cdc2 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -448,21 +448,16 @@ static ColumnPtr andFilters(ColumnPtr c1, ColumnPtr c2) throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of filters don't match: {} and {}", c1->size(), c2->size()); - // TODO: use proper vectorized implementation of AND? auto res = ColumnUInt8::create(c1->size()); auto & res_data = res->getData(); const auto & c1_data = typeid_cast(*c1).getData(); const auto & c2_data = typeid_cast(*c2).getData(); const size_t size = c1->size(); - const size_t step = 16; - size_t i = 0; - /// NOTE: '&&' must be used instead of '&' for 'AND' operation because UInt8 columns might contain any non-zero - /// value for true and we cannot bitwise AND them to get the correct result. - for (; i + step < size; i += step) - for (size_t j = 0; j < step; ++j) - res_data[i+j] = (c1_data[i+j] && c2_data[i+j]); - for (; i < size; ++i) - res_data[i] = (c1_data[i] && c2_data[i]); + /// The double NOT operators (!!) convert the non-zeros to the bool value of true (0x01) and zeros to false (0x00). + /// After casting them to UInt8, '&' could replace '&&' for the 'AND' operation implementation and at the same + /// time enable the auto vectorization. + for (size_t i = 0; i < size; ++i) + res_data[i] = (static_cast(!!c1_data[i]) & static_cast(!!c2_data[i])); return res; } diff --git a/src/Storages/MergeTree/tests/gtest_combine_filters.cpp b/src/Storages/MergeTree/tests/gtest_combine_filters.cpp index 53696474eb8..9a9e6caad81 100644 --- a/src/Storages/MergeTree/tests/gtest_combine_filters.cpp +++ b/src/Storages/MergeTree/tests/gtest_combine_filters.cpp @@ -138,6 +138,57 @@ bool testCombineColumns(size_t size) return true; } +/* To ensure the vectorized DB::andFilters works as its scalar implementation, this test validates the AND (&&) + * of any combinations of the UInt8 values. + */ +bool testAndFilters(size_t size) +{ + auto generateFastIncrementColumn = [](size_t len)->ColumnPtr + { + auto filter = ColumnUInt8::create(len); + auto & filter_data = filter->getData(); + + for (size_t i = 0; i < len; ++i) + filter_data[i] = static_cast(i & 0xFF); + + return filter; + }; + + auto generateSlowIncrementColumn = [](size_t len)->ColumnPtr + { + auto filter = ColumnUInt8::create(len); + auto & filter_data = filter->getData(); + + for (size_t i = 0; i < len; ++i) + filter_data[i] = static_cast((i >> 8) & 0xFF); + + return filter; + }; + + auto first_filter = generateFastIncrementColumn(size); + auto second_filter = generateSlowIncrementColumn(size); + + auto result = andFilters(first_filter, second_filter); + + const auto & first_filter_data = typeid_cast(first_filter.get())->getData(); + const auto & second_filter_data = typeid_cast(second_filter.get())->getData(); + const auto & result_data = typeid_cast(result.get())->getData(); + + if (result->size() != size) + { + return false; + } + + for (size_t i = 0; i < size; i++) + { + UInt8 expected = first_filter_data[i] && second_filter_data[i]; + if (result_data[i] != expected) + return false; + } + + return true; +} + TEST(MergeTree, CombineFilters) { /// Tests with only 0/1 and fixed intervals. @@ -159,3 +210,18 @@ TEST(MergeTree, CombineFilters) EXPECT_TRUE(testCombineColumns(2000)); EXPECT_TRUE(testCombineColumns(200000)); } + +TEST(MergeTree, AndFilters) +{ + EXPECT_TRUE(testAndFilters(1)); + EXPECT_TRUE(testAndFilters(2)); + EXPECT_TRUE(testAndFilters(15)); + EXPECT_TRUE(testAndFilters(16)); + EXPECT_TRUE(testAndFilters(200)); + EXPECT_TRUE(testAndFilters(201)); + EXPECT_TRUE(testAndFilters(2000)); + EXPECT_TRUE(testAndFilters(65535)); + EXPECT_TRUE(testAndFilters(65536)); + EXPECT_TRUE(testAndFilters(65537)); + EXPECT_TRUE(testAndFilters(200000)); +} From 37a3b10c42559af656d2cfce63ca27b00163947b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 28 Feb 2024 15:17:27 +0100 Subject: [PATCH 20/60] Speedup check-whitespaces --- utils/check-style/check-style | 5 ---- utils/check-style/check-whitespaces | 2 +- utils/check-style/double-whitespaces.pl | 36 ++++++++++++++----------- 3 files changed, 21 insertions(+), 22 deletions(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index be1d3c373f9..7b0b9b1209b 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -50,11 +50,6 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/n # Broken symlinks find -L $ROOT_PATH -type l 2>/dev/null | grep -v contrib && echo "^ Broken symlinks found" -# Double whitespaces -find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null | - grep -vP $EXCLUDE_DIRS | - while read i; do $ROOT_PATH/utils/check-style/double-whitespaces.pl < $i || echo -e "^ File $i contains double whitespaces\n"; done - # Unused/Undefined/Duplicates ErrorCodes/ProfileEvents/CurrentMetrics declare -A EXTERN_TYPES EXTERN_TYPES[ErrorCodes]=int diff --git a/utils/check-style/check-whitespaces b/utils/check-style/check-whitespaces index 5a20569868d..eaf1750bf6a 100755 --- a/utils/check-style/check-whitespaces +++ b/utils/check-style/check-whitespaces @@ -6,4 +6,4 @@ EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|c # Double whitespaces find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null | grep -vP $EXCLUDE_DIRS | - while read i; do $ROOT_PATH/utils/check-style/double-whitespaces.pl < $i || echo -e "^ File $i contains double whitespaces\n"; done + parallel -m "${ROOT_PATH}/utils/check-style/double-whitespaces.pl" diff --git a/utils/check-style/double-whitespaces.pl b/utils/check-style/double-whitespaces.pl index 47b03cb74ab..daeddecbd27 100755 --- a/utils/check-style/double-whitespaces.pl +++ b/utils/check-style/double-whitespaces.pl @@ -5,27 +5,31 @@ use strict; # Find double whitespace such as "a, b, c" that looks very ugly and annoying. # But skip double whitespaces if they are used as an alignment - by comparing to surrounding lines. -my @array; - -while (<>) -{ - push @array, $_; -} - my $ret = 0; -for (my $i = 1; $i < $#array; ++$i) +foreach my $file (@ARGV) { - if ($array[$i] =~ ',( {2,3})[^ /]') - { - # https://stackoverflow.com/questions/87380/how-can-i-find-the-location-of-a-regex-match-in-perl + my @array; - if ((substr($array[$i - 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) # whitespaces are not part of alignment - && (substr($array[$i + 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) - && $array[$i] !~ /(-?\d+\w*,\s+){3,}/) # this is not a number table like { 10, -1, 2 } + open (FH,'<',$file); + while () + { + push @array, $_; + } + + for (my $i = 1; $i < $#array; ++$i) + { + if ($array[$i] =~ ',( {2,3})[^ /]') { - print(($i + 1) . ":" . $array[$i]); - $ret = 1; + # https://stackoverflow.com/questions/87380/how-can-i-find-the-location-of-a-regex-match-in-perl + + if ((substr($array[$i - 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) # whitespaces are not part of alignment + && (substr($array[$i + 1], $+[1] - 1, 2) !~ /^[ -][^ ]$/) + && $array[$i] !~ /(-?\d+\w*,\s+){3,}/) # this is not a number table like { 10, -1, 2 } + { + print($file . ":" . ($i + 1) . $array[$i]); + $ret = 1; + } } } } From c22da4c2e8f82106027c7452112fa0c3461ced33 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 28 Feb 2024 15:59:17 +0100 Subject: [PATCH 21/60] Try -X instead --- utils/check-style/check-whitespaces | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-whitespaces b/utils/check-style/check-whitespaces index eaf1750bf6a..ad34ff3aab0 100755 --- a/utils/check-style/check-whitespaces +++ b/utils/check-style/check-whitespaces @@ -6,4 +6,4 @@ EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|c # Double whitespaces find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null | grep -vP $EXCLUDE_DIRS | - parallel -m "${ROOT_PATH}/utils/check-style/double-whitespaces.pl" + parallel -X "${ROOT_PATH}/utils/check-style/double-whitespaces.pl" From f6294a5c8b11793bfe2c2aa4563c32876d5635d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 28 Feb 2024 16:29:07 +0100 Subject: [PATCH 22/60] I dislike Ubuntu --- utils/check-style/check-whitespaces | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-whitespaces b/utils/check-style/check-whitespaces index ad34ff3aab0..507b1dd2ede 100755 --- a/utils/check-style/check-whitespaces +++ b/utils/check-style/check-whitespaces @@ -2,8 +2,9 @@ ROOT_PATH=$(git rev-parse --show-toplevel) EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing/|Parsers/New' +NPROC=$(($(nproc) + 3)) # Double whitespaces find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null | grep -vP $EXCLUDE_DIRS | - parallel -X "${ROOT_PATH}/utils/check-style/double-whitespaces.pl" + xargs -P "$NPROC" -n 20 "${ROOT_PATH}/utils/check-style/double-whitespaces.pl" From 5d68c9f0465d3fb801e3f836ecd2d63f097fecf4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 29 Feb 2024 09:38:13 +0100 Subject: [PATCH 23/60] Updated default value of endpoint_contains_account_name to true --- .../en/engines/table-engines/integrations/azureBlobStorage.md | 2 +- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp | 4 ++-- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 2 +- tests/integration/test_merge_tree_azure_blob_storage/test.py | 1 + 5 files changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index 70d696f9684..0843ff1ac47 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -20,7 +20,7 @@ CREATE TABLE azure_blob_storage_table (name String, value UInt32) ### Engine parameters - `endpoint` — AzureBlobStorage endpoint URL with container & prefix. Optionally can contain account_name if the authentication method used needs it. (http://azurite1:{port}/[account_name]{container_name}/{data_prefix}) or these parameters can be provided separately using storage_account_url, account_name & container. For specifying prefix, endpoint should be used. -- `endpoint_contains_account_name` - This flag is used to specify if endpoint contains account_name as it is only needed for certain authentication methods. +- `endpoint_contains_account_name` - This flag is used to specify if endpoint contains account_name as it is only needed for certain authentication methods. (Default : true) - `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) - `container_name` - Container name - `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 5f66c2efecd..a2c54cb412b 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1237,7 +1237,7 @@ Configuration markup: Connection parameters: * `endpoint` — AzureBlobStorage endpoint URL with container & prefix. Optionally can contain account_name if the authentication method used needs it. (`http://account.blob.core.windows.net:{port}/[account_name]{container_name}/{data_prefix}`) or these parameters can be provided separately using storage_account_url, account_name & container. For specifying prefix, endpoint should be used. -* `endpoint_contains_account_name` - This flag is used to specify if endpoint contains account_name as it is only needed for certain authentication methods. (Default : false) +* `endpoint_contains_account_name` - This flag is used to specify if endpoint contains account_name as it is only needed for certain authentication methods. (Default : true) * `storage_account_url` - Required if endpoint is not specified, Azure Blob Storage account URL, like `http://account.blob.core.windows.net` or `http://azurite1:10000/devstoreaccount1`. * `container_name` - Target container name, defaults to `default-container`. * `container_already_exists` - If set to `false`, a new container `container_name` is created in the storage account, if set to `true`, disk connects to the container directly, and if left unset, disk connects to the account, checks if the container `container_name` exists, and creates it if it doesn't exist yet. diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 7dfee3bc760..d5815795744 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -60,8 +60,8 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr String endpoint = config.getString(config_prefix + ".endpoint"); /// For some authentication methods account name is not present in the endpoint - /// 'endpoint_contains_account_name' bool is used to understand how to split the endpoint (default : false) - bool endpoint_contains_account_name = config.getBool(config_prefix + ".endpoint_contains_account_name", false); + /// 'endpoint_contains_account_name' bool is used to understand how to split the endpoint (default : true) + bool endpoint_contains_account_name = config.getBool(config_prefix + ".endpoint_contains_account_name", true); size_t pos = endpoint.find("//"); if (pos == std::string::npos) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 8109b8eaf54..55d2f228bf0 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -67,7 +67,7 @@ public: const String & name_, AzureClientPtr && client_, SettingsPtr && settings_, - const String & container_); + const String & object_namespace_); void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 27aa6c12610..55deb87a97e 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -740,6 +740,7 @@ def test_endpoint_error_check(cluster): SETTINGS disk = disk( type = azure_blob_storage, endpoint = 'http://azurite1:{port}', + endpoint_contains_account_name = 'false', account_name = 'devstoreaccount1', account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', skip_access_check = 0); From eb4ec0e871e966742812a65f5142310b7d3c0602 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Fri, 19 Jan 2024 07:05:36 -0800 Subject: [PATCH 24/60] Add support for reading and writing backups as a tar archive using libarchive. --- src/Backups/BackupImpl.cpp | 2 +- src/IO/Archives/IArchiveWriter.h | 3 + src/IO/Archives/LibArchiveReader.cpp | 156 +++++++--- src/IO/Archives/LibArchiveReader.h | 13 +- src/IO/Archives/LibArchiveWriter.cpp | 267 ++++++++++++++++++ src/IO/Archives/LibArchiveWriter.h | 76 +++++ src/IO/Archives/ZipArchiveWriter.cpp | 5 + src/IO/Archives/ZipArchiveWriter.h | 3 + src/IO/Archives/createArchiveReader.cpp | 2 +- src/IO/Archives/createArchiveWriter.cpp | 21 +- .../hasRegisteredArchiveFileExtension.cpp | 2 +- .../tests/gtest_archive_reader_and_writer.cpp | 120 ++++++-- .../test_backup_restore_new/test.py | 51 ++++ .../test_backup_restore_s3/test.py | 5 + 14 files changed, 651 insertions(+), 75 deletions(-) create mode 100644 src/IO/Archives/LibArchiveWriter.cpp create mode 100644 src/IO/Archives/LibArchiveWriter.h diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 42fdb719149..8c0989b8202 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -927,7 +927,7 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) const auto write_info_to_archive = [&](const auto & file_name) { - auto out = archive_writer->writeFile(file_name); + auto out = archive_writer->writeFile(file_name, info.size); auto read_buffer = entry->getReadBuffer(writer->getReadSettings()); if (info.base_size != 0) read_buffer->seek(info.base_size, SEEK_SET); diff --git a/src/IO/Archives/IArchiveWriter.h b/src/IO/Archives/IArchiveWriter.h index cccc6dc953b..3dce83629fb 100644 --- a/src/IO/Archives/IArchiveWriter.h +++ b/src/IO/Archives/IArchiveWriter.h @@ -22,6 +22,9 @@ public: /// of the function `writeFile()` should be destroyed before next call of `writeFile()`. virtual std::unique_ptr writeFile(const String & filename) = 0; + virtual std::unique_ptr writeFile(const String & filename, const size_t & size) = 0; + + /// Returns true if there is an active instance of WriteBuffer returned by writeFile(). /// This function should be used mostly for debugging purposes. virtual bool isWritingFile() const = 0; diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 22300ed5444..83f34f31568 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -5,7 +5,6 @@ #include -#include namespace DB { @@ -14,35 +13,60 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_UNPACK_ARCHIVE; - extern const int LOGICAL_ERROR; - extern const int CANNOT_READ_ALL_DATA; - extern const int UNSUPPORTED_METHOD; +extern const int CANNOT_UNPACK_ARCHIVE; +extern const int LOGICAL_ERROR; +extern const int CANNOT_READ_ALL_DATA; +extern const int UNSUPPORTED_METHOD; } +class LibArchiveReader::StreamInfo +{ +public: + explicit StreamInfo(std::unique_ptr read_buffer_) + : read_buffer(std::move(read_buffer_)) + { + } + + static ssize_t read([[maybe_unused]] struct archive * a, void * client_data, const void ** buff) + { + auto * read_stream = reinterpret_cast(client_data); + *buff = reinterpret_cast(read_stream->buf); + return read_stream->read_buffer->read(read_stream->buf, DBMS_DEFAULT_BUFFER_SIZE); + } + + std::unique_ptr read_buffer; + char buf[DBMS_DEFAULT_BUFFER_SIZE]; +}; + class LibArchiveReader::Handle { public: explicit Handle(std::string path_to_archive_, bool lock_on_reading_) - : path_to_archive(path_to_archive_), lock_on_reading(lock_on_reading_) + : path_to_archive(path_to_archive_), lock_on_reading(lock_on_reading_) { - current_archive = open(path_to_archive); + current_archive = openWithPath(path_to_archive); + } + explicit Handle(std::string path_to_archive_, bool lock_on_reading_, const ReadArchiveFunction & archive_read_function_) + : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), lock_on_reading(lock_on_reading_) + { + read_stream = std::make_unique(archive_read_function()); + current_archive = openWithReader(&(*read_stream)); } Handle(const Handle &) = delete; Handle(Handle && other) noexcept : current_archive(other.current_archive) , current_entry(other.current_entry) + , read_stream(std::move(other.read_stream)) + , archive_read_function(std::move(other.archive_read_function)) , lock_on_reading(other.lock_on_reading) + { other.current_archive = nullptr; other.current_entry = nullptr; } - ~Handle() - { - close(current_archive); - } + ~Handle() { close(current_archive); } bool locateFile(const std::string & filename) { @@ -64,10 +88,14 @@ public: break; if (filter(archive_entry_pathname(current_entry))) + { + valid = true; return true; + } } checkError(err); + valid = false; return false; } @@ -81,12 +109,24 @@ public: } while (err == ARCHIVE_RETRY); checkError(err); - return err == ARCHIVE_OK; + valid = err == ARCHIVE_OK; + return valid; } std::vector getAllFiles(NameFilter filter) { - auto * archive = open(path_to_archive); + struct archive * archive; + std::unique_ptr rs; + if(archive_read_function) + { + read_stream = std::make_unique(archive_read_function()); + archive = openWithReader(&(*rs)); + } + else + { + archive = openWithPath(path_to_archive); + } + SCOPE_EXIT( close(archive); ); @@ -94,7 +134,7 @@ public: struct archive_entry * entry = nullptr; std::vector files; - int error = readNextHeader(archive, &entry); + int error = readNextHeader(current_archive, &entry); while (error == ARCHIVE_OK || error == ARCHIVE_RETRY) { chassert(entry != nullptr); @@ -102,7 +142,7 @@ public: if (!filter || filter(name)) files.push_back(std::move(name)); - error = readNextHeader(archive, &entry); + error = readNextHeader(current_archive, &entry); } checkError(error); @@ -112,6 +152,8 @@ public: const String & getFileName() const { chassert(current_entry); + if (!valid) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "No current file"); if (!file_name) file_name.emplace(archive_entry_pathname(current_entry)); @@ -121,6 +163,8 @@ public: const FileInfo & getFileInfo() const { chassert(current_entry); + if (!valid) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "No current file"); if (!file_info) { file_info.emplace(); @@ -134,6 +178,8 @@ public: struct archive * current_archive; struct archive_entry * current_entry = nullptr; + bool valid = true; + private: void checkError(int error) const { @@ -147,7 +193,16 @@ private: file_info.reset(); } - static struct archive * open(const String & path_to_archive) + static struct archive * openWithReader(StreamInfo * read_stream_) + { + auto * a = archive_read_new(); + archive_read_support_filter_all(a); + archive_read_support_format_all(a); + archive_read_open(a, read_stream_, nullptr, StreamInfo::read, nullptr); + return a; + } + + static struct archive * openWithPath(const String & path_to_archive) { auto * archive = archive_read_new(); try @@ -194,6 +249,8 @@ private: } const String path_to_archive; + std::unique_ptr read_stream = nullptr; + const IArchiveReader::ReadArchiveFunction archive_read_function; /// for some archive types when we are reading headers static variables are used /// which are not thread-safe @@ -207,7 +264,7 @@ private: class LibArchiveReader::FileEnumeratorImpl : public FileEnumerator { public: - explicit FileEnumeratorImpl(Handle handle_) : handle(std::move(handle_)) {} + explicit FileEnumeratorImpl(Handle handle_) : handle(std::move(handle_)) { } const String & getFileName() const override { return handle.getFileName(); } const FileInfo & getFileInfo() const override { return handle.getFileInfo(); } @@ -215,6 +272,7 @@ public: /// Releases owned handle to pass it to a read buffer. Handle releaseHandle() && { return std::move(handle); } + private: Handle handle; }; @@ -226,13 +284,13 @@ public: : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) , handle(std::move(handle_)) , path_to_archive(std::move(path_to_archive_)) - {} + { + } off_t seek(off_t /* off */, int /* whence */) override { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Seek is not supported when reading from archive"); } - bool checkIfActuallySeekable() override { return false; } off_t getPosition() override @@ -240,14 +298,13 @@ public: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); } + off_t getPosition() override { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); } + String getFileName() const override { return handle.getFileName(); } size_t getFileSize() override { return handle.getFileInfo().uncompressed_size; } - Handle releaseHandle() && - { - return std::move(handle); - } + Handle releaseHandle() && { return std::move(handle); } private: bool nextImpl() override @@ -274,7 +331,17 @@ private: LibArchiveReader::LibArchiveReader(std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_) : archive_name(std::move(archive_name_)), lock_on_reading(lock_on_reading_), path_to_archive(std::move(path_to_archive_)) -{} +{ +} + +LibArchiveReader::LibArchiveReader( + std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_, const ReadArchiveFunction & archive_read_function_) + : archive_name(std::move(archive_name_)) + , lock_on_reading(lock_on_reading_) + , path_to_archive(std::move(path_to_archive_)) + , archive_read_function(archive_read_function_) +{ +} LibArchiveReader::~LibArchiveReader() = default; @@ -285,21 +352,21 @@ const std::string & LibArchiveReader::getPath() const bool LibArchiveReader::fileExists(const String & filename) { - Handle handle(path_to_archive, lock_on_reading); + Handle handle = acquireHandle(); return handle.locateFile(filename); } LibArchiveReader::FileInfo LibArchiveReader::getFileInfo(const String & filename) { - Handle handle(path_to_archive, lock_on_reading); + Handle handle = acquireHandle(); if (!handle.locateFile(filename)) - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: file not found", path_to_archive); + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: File {} was not found in archive", path_to_archive, quoteString(filename)); return handle.getFileInfo(); } std::unique_ptr LibArchiveReader::firstFile() { - Handle handle(path_to_archive, lock_on_reading); + Handle handle = acquireHandle(); if (!handle.nextFile()) return nullptr; @@ -308,17 +375,25 @@ std::unique_ptr LibArchiveReader::firstFile() std::unique_ptr LibArchiveReader::readFile(const String & filename, bool throw_on_not_found) { - return readFile([&](const std::string & file) { return file == filename; }, throw_on_not_found); + Handle handle = acquireHandle(); + if (!handle.locateFile(filename)) + { + if (throw_on_not_found) + throw Exception( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: File {} was not found in archive", path_to_archive, quoteString(filename)); + return nullptr; + } + return std::make_unique(std::move(handle), path_to_archive); } std::unique_ptr LibArchiveReader::readFile(NameFilter filter, bool throw_on_not_found) { - Handle handle(path_to_archive, lock_on_reading); + Handle handle = acquireHandle(); if (!handle.locateFile(filter)) { if (throw_on_not_found) throw Exception( - ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: no file found satisfying the filter", path_to_archive); + ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: No file satisfying filter in archive", path_to_archive); return nullptr; } return std::make_unique(std::move(handle), path_to_archive); @@ -337,7 +412,8 @@ std::unique_ptr LibArchiveReader::nextFile(std { if (!dynamic_cast(read_buffer.get())) throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong ReadBuffer passed to nextFile()"); - auto read_buffer_from_libarchive = std::unique_ptr(static_cast(read_buffer.release())); + auto read_buffer_from_libarchive + = std::unique_ptr(static_cast(read_buffer.release())); auto handle = std::move(*read_buffer_from_libarchive).releaseHandle(); if (!handle.nextFile()) return nullptr; @@ -360,13 +436,23 @@ std::vector LibArchiveReader::getAllFiles() std::vector LibArchiveReader::getAllFiles(NameFilter filter) { - Handle handle(path_to_archive, lock_on_reading); + Handle handle = acquireHandle(); return handle.getAllFiles(filter); } -void LibArchiveReader::setPassword(const String & /*password_*/) +void LibArchiveReader::setPassword([[maybe_unused]] const String & password_) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to {} archive", archive_name); + if (password_ != "") + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to {} archive", archive_name); +} + +LibArchiveReader::Handle LibArchiveReader::acquireHandle() +{ + std::lock_guard lock{mutex}; + if (archive_read_function) + return Handle{path_to_archive, lock_on_reading, archive_read_function}; + else + return Handle{path_to_archive, lock_on_reading}; } #endif diff --git a/src/IO/Archives/LibArchiveReader.h b/src/IO/Archives/LibArchiveReader.h index c4b08d8ddf7..cca72e89d4d 100644 --- a/src/IO/Archives/LibArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -1,7 +1,7 @@ #pragma once #include "config.h" - +#include #include @@ -52,20 +52,31 @@ protected: /// Constructs an archive's reader that will read from a file in the local filesystem. LibArchiveReader(std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_); + LibArchiveReader(std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_, const ReadArchiveFunction & archive_read_function_); + private: class ReadBufferFromLibArchive; class Handle; class FileEnumeratorImpl; + class StreamInfo; + + Handle acquireHandle(); const std::string archive_name; const bool lock_on_reading; const String path_to_archive; + const ReadArchiveFunction archive_read_function; + mutable std::mutex mutex; + }; class TarArchiveReader : public LibArchiveReader { public: explicit TarArchiveReader(std::string path_to_archive) : LibArchiveReader("tar", /*lock_on_reading_=*/ true, std::move(path_to_archive)) { } + + explicit TarArchiveReader(std::string path_to_archive, const ReadArchiveFunction & archive_read_function ) : LibArchiveReader("tar", /*lock_on_reading_=*/ true, std::move(path_to_archive), archive_read_function) { } + }; class SevenZipArchiveReader : public LibArchiveReader diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp new file mode 100644 index 00000000000..c9a34cd014b --- /dev/null +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -0,0 +1,267 @@ +#include + +#include +#include +#include +#include + +#include + +#if USE_LIBARCHIVE + +// this implemation follows the ZipArchiveWriter implemation as closely as possible. + +namespace DB +{ +namespace ErrorCodes +{ +extern const int CANNOT_PACK_ARCHIVE; +extern const int LOGICAL_ERROR; +extern const int CANNOT_READ_ALL_DATA; +extern const int UNSUPPORTED_METHOD; +extern const int NOT_IMPLEMENTED; +} + +namespace +{ +void checkResultCodeImpl(int code, const String & filename) +{ + if (code == ARCHIVE_OK) + return; + String message = "LibArchive Code = " + std::to_string(code); + throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't pack archive: {}, filename={}", message, quoteString(filename)); +} +} + +// this is a thin wrapper for libarchive to be able to write the archive to a WriteBuffer +class LibArchiveWriter::StreamInfo +{ +public: + explicit StreamInfo(std::unique_ptr archive_write_buffer_) : archive_write_buffer(std::move(archive_write_buffer_)) { } + + + static ssize_t memory_write([[maybe_unused]] struct archive * a, void * client_data, const void * buff, size_t length) + { + auto * stream_info = reinterpret_cast(client_data); + stream_info->archive_write_buffer->write(reinterpret_cast(buff), length); + return length; + } + + std::unique_ptr archive_write_buffer; +}; + +class LibArchiveWriter::WriteBufferFromLibArchive : public WriteBufferFromFileBase +{ +public: + WriteBufferFromLibArchive(std::shared_ptr archive_writer_, const String & filename_, const size_t & size_) + : WriteBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0), archive_writer(archive_writer_), filename(filename_), size(size_) + { + startWritingFile(); + a = archive_writer_->getArchive(); + entry = nullptr; + } + + + ~WriteBufferFromLibArchive() override + { + try + { + closeFile(/* throw_if_error= */ false); + endWritingFile(); + } + catch (...) + { + tryLogCurrentException("WriteBufferFromTarArchive"); + } + } + + void finalizeImpl() override + { + next(); + closeFile(/* throw_if_error=*/true); + endWritingFile(); + } + + void sync() override { next(); } + std::string getFileName() const override { return filename; } + + +private: + void nextImpl() override + { + if (!offset()) + return; + if (entry == nullptr) + writeEntry(); + ssize_t to_write = offset(); + ssize_t written = archive_write_data(a, working_buffer.begin(), offset()); + if (written != to_write) + { + throw Exception( + ErrorCodes::CANNOT_PACK_ARCHIVE, + "Couldn't pack tar archive: Failed to write all bytes, {} of {} , filename={}", + written, + to_write, + quoteString(filename)); + } + } + + + void writeEntry() + { + expected_size = getSize(); + entry = archive_entry_new(); + archive_entry_set_pathname(entry, filename.c_str()); + archive_entry_set_size(entry, expected_size); + archive_entry_set_filetype(entry, static_cast<__LA_MODE_T>(0100000)); + archive_entry_set_perm(entry, 0644); + checkResult(archive_write_header(a, entry)); + } + + size_t getSize() const + { + if (size) + return size; + else + return offset(); + } + + void closeFile([[maybe_unused]] bool throw_if_error) + { + if (entry) + { + archive_entry_free(entry); + entry = nullptr; + } + if (throw_if_error and bytes != expected_size) + { + throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't pack tar archive: Wrote {} of expected {} , filename={}", bytes, expected_size, quoteString(filename)); + } + } + + void endWritingFile() + { + if (auto archive_writer_ptr = archive_writer.lock()) + archive_writer_ptr->endWritingFile(); + } + + void startWritingFile() + { + if (auto archive_writer_ptr = archive_writer.lock()) + archive_writer_ptr->startWritingFile(); + } + + void checkResult(int code) { checkResultCodeImpl(code, filename); } + + std::weak_ptr archive_writer; + const String filename; + struct archive_entry * entry; + struct archive * a; + size_t size; + size_t expected_size; +}; + +LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_) : path_to_archive(path_to_archive_) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not Implemented"); +} + +LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_) + : path_to_archive(path_to_archive_) +{ + a = archive_write_new(); + archive_write_set_format_pax_restricted(a); + //this allows use to write directly to a writer buffer rather than an intermediate buffer in LibArchive + //archive_write_set_bytes_per_block(a, 0); + if (archive_write_buffer_) + { + stream_info = std::make_unique(std::move(archive_write_buffer_)); + archive_write_open2(a, &(*stream_info), nullptr, &StreamInfo::memory_write, nullptr, nullptr); + } + else + { + archive_write_open_filename(a, path_to_archive.c_str()); + } +} + + +LibArchiveWriter::~LibArchiveWriter() +{ + if (!finalized) + { + if (!std::uncaught_exceptions() && std::current_exception() == nullptr) + chassert(false && "TarArchiveWriter is not finalized in destructor."); + } + + if (a) + archive_write_free(a); +} + +std::unique_ptr LibArchiveWriter::writeFile(const String & filename, const size_t & size) +{ + return std::make_unique(std::static_pointer_cast(shared_from_this()), filename, size); +} + +std::unique_ptr LibArchiveWriter::writeFile(const String & filename) +{ + return std::make_unique(std::static_pointer_cast(shared_from_this()), filename, 0); +} + +bool LibArchiveWriter::isWritingFile() const +{ + std::lock_guard lock{mutex}; + return is_writing_file; +} + +void LibArchiveWriter::endWritingFile() +{ + std::lock_guard lock{mutex}; + is_writing_file = false; +} + +void LibArchiveWriter::startWritingFile() +{ + std::lock_guard lock{mutex}; + if (is_writing_file) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot write two files to a tar archive in parallel"); + is_writing_file = true; +} + +void LibArchiveWriter::finalize() +{ + std::lock_guard lock{mutex}; + if (finalized) + return; + if (a) + archive_write_close(a); + if (stream_info) + { + stream_info->archive_write_buffer->finalize(); + stream_info.reset(); + } + finalized = true; +} + +void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) +{ + if (compression_method_.size() == 0 and compression_level == 1) + { + return; + } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Tar archives are currenly supported without compression"); +} + +void LibArchiveWriter::setPassword([[maybe_unused]] const String & password_) +{ + if (password_ == "") + return; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for tar archives"); +} + +struct archive * LibArchiveWriter::getArchive() +{ + std::lock_guard lock{mutex}; + return a; +} +} +#endif diff --git a/src/IO/Archives/LibArchiveWriter.h b/src/IO/Archives/LibArchiveWriter.h new file mode 100644 index 00000000000..982265cb2f1 --- /dev/null +++ b/src/IO/Archives/LibArchiveWriter.h @@ -0,0 +1,76 @@ +#pragma once + +#include "config.h" + +#if USE_LIBARCHIVE +# include +# include +# include +# include + + +namespace DB +{ +class WriteBufferFromFileBase; + +/// Interface for writing an archive. +class LibArchiveWriter : public IArchiveWriter +{ +public: + /// Constructs an archive that will be written as a file in the local filesystem. + [[noreturn]] explicit LibArchiveWriter(const String & path_to_archive_); + + /// Constructs an archive that will be written as a file in the local filesystem. + explicit LibArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_); + + /// Call finalize() before destructing IArchiveWriter. + ~LibArchiveWriter() override; + + /// Starts writing a file to the archive. The function returns a write buffer, + /// any data written to that buffer will be compressed and then put to the archive. + /// You can keep only one such buffer at a time, a buffer returned by previous call + /// of the function `writeFile()` should be destroyed before next call of `writeFile()`. + std::unique_ptr writeFile(const String & filename) override; + /// LibArchive needs to know the size of the file being written. If the file size is not + /// passed in the the archive writer tries to infer the size by looking at the available + /// data in the buffer, if next is called before all data is written to the buffer + /// an exception is thrown. + std::unique_ptr writeFile(const String & filename, const size_t & size) override; + + + /// Returns true if there is an active instance of WriteBuffer returned by writeFile(). + /// This function should be used mostly for debugging purposes. + bool isWritingFile() const override; + + /// Finalizes writing of the archive. This function must be always called at the end of writing. + /// (Unless an error appeared and the archive is in fact no longer needed.) + void finalize() override; + + static constexpr const int kDefaultCompressionLevel = -1; + + /// Sets compression method and level. + /// Changing them will affect next file in the archive. + void setCompression(const String & /* compression_method */, int /* compression_level */ = kDefaultCompressionLevel) override; + + /// Sets password. If the password is not empty it will enable encryption in the archive. + void setPassword(const String & /* password */) override; + +private: + class WriteBufferFromLibArchive; + class StreamInfo; + + struct archive * getArchive(); + void startWritingFile(); + void endWritingFile(); + + String path_to_archive; + std::unique_ptr stream_info TSA_GUARDED_BY(mutex) = nullptr; + struct archive * a TSA_GUARDED_BY(mutex) = nullptr; + bool is_writing_file TSA_GUARDED_BY(mutex) = false; + bool finalized TSA_GUARDED_BY(mutex) = false; + mutable std::mutex mutex; +}; + +} + +#endif diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 8cb4a2e0bd6..521e1e4e74c 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -274,6 +274,11 @@ std::unique_ptr ZipArchiveWriter::writeFile(const Strin return std::make_unique(std::static_pointer_cast(shared_from_this()), filename); } +std::unique_ptr ZipArchiveWriter::writeFile(const String & filename, [[maybe_unused]] const size_t & size) +{ + return ZipArchiveWriter::writeFile(filename); +} + bool ZipArchiveWriter::isWritingFile() const { std::lock_guard lock{mutex}; diff --git a/src/IO/Archives/ZipArchiveWriter.h b/src/IO/Archives/ZipArchiveWriter.h index 891da1a2e75..0b8260b1f2f 100644 --- a/src/IO/Archives/ZipArchiveWriter.h +++ b/src/IO/Archives/ZipArchiveWriter.h @@ -32,6 +32,9 @@ public: /// of the function `writeFile()` should be destroyed before next call of `writeFile()`. std::unique_ptr writeFile(const String & filename) override; + std::unique_ptr writeFile(const String & filename, const size_t & size) override; + + /// Returns true if there is an active instance of WriteBuffer returned by writeFile(). /// This function should be used mostly for debugging purposes. bool isWritingFile() const override; diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index 0c998971de1..03623c78624 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -48,7 +48,7 @@ std::shared_ptr createArchiveReader( tar_extensions.begin(), tar_extensions.end(), [&](const auto extension) { return path_to_archive.ends_with(extension); })) { #if USE_LIBARCHIVE - return std::make_shared(path_to_archive); + return std::make_shared(path_to_archive, archive_read_function); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); #endif diff --git a/src/IO/Archives/createArchiveWriter.cpp b/src/IO/Archives/createArchiveWriter.cpp index 807fe66e6a9..2fb56e14bd3 100644 --- a/src/IO/Archives/createArchiveWriter.cpp +++ b/src/IO/Archives/createArchiveWriter.cpp @@ -1,5 +1,6 @@ -#include +#include #include +#include #include #include @@ -8,8 +9,8 @@ namespace DB { namespace ErrorCodes { - extern const int CANNOT_PACK_ARCHIVE; - extern const int SUPPORT_IS_DISABLED; +extern const int CANNOT_PACK_ARCHIVE; +extern const int SUPPORT_IS_DISABLED; } @@ -19,9 +20,8 @@ std::shared_ptr createArchiveWriter(const String & path_to_archi } -std::shared_ptr createArchiveWriter( - const String & path_to_archive, - [[maybe_unused]] std::unique_ptr archive_write_buffer) +std::shared_ptr +createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique_ptr archive_write_buffer) { if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) { @@ -29,6 +29,15 @@ std::shared_ptr createArchiveWriter( return std::make_shared(path_to_archive, std::move(archive_write_buffer)); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); +#endif + } + //todo add support for extentsions i.e .gz + else if (path_to_archive.ends_with(".tar")) + { +#if USE_LIBARCHIVE + return std::make_shared(path_to_archive, std::move(archive_write_buffer)); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); #endif } else diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp index 6b2ef29d054..3ac9bba5e5d 100644 --- a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp @@ -6,7 +6,7 @@ namespace DB bool hasRegisteredArchiveFileExtension(const String & path) { - return path.ends_with(".zip") || path.ends_with(".zipx"); + return path.ends_with(".zip") || path.ends_with(".zipx") || path.ends_with(".tar"); } } diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index 37fbdff901a..e5909ce5869 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -7,9 +7,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -335,41 +337,51 @@ TEST_P(ArchiveReaderAndWriterTest, InMemory) TEST_P(ArchiveReaderAndWriterTest, Password) -{ - /// Make an archive. - std::string_view contents = "The contents of a.txt"; +{ + auto writer = createArchiveWriter(getPathToArchive()); + //don't support passwords for tar archives + if(getPathToArchive().ends_with(".tar")) { - auto writer = createArchiveWriter(getPathToArchive()); - writer->setPassword("Qwe123"); - { - auto out = writer->writeFile("a.txt"); - writeString(contents, *out); - out->finalize(); - } + expectException(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for tar archives", + [&]{ writer->setPassword("a.txt"); }); writer->finalize(); } - - /// Read the archive. - auto reader = createArchiveReader(getPathToArchive()); - - /// Try to read without a password. - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Password is required", - [&]{ reader->readFile("a.txt", /*throw_on_not_found=*/true); }); - + else { - /// Try to read with a wrong password. - reader->setPassword("123Qwe"); - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Wrong password", + /// Make an archive. + std::string_view contents = "The contents of a.txt"; + { + writer->setPassword("Qwe123"); + { + auto out = writer->writeFile("a.txt"); + writeString(contents, *out); + out->finalize(); + } + writer->finalize(); + } + + /// Read the archive. + auto reader = createArchiveReader(getPathToArchive()); + + /// Try to read without a password. + expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Password is required", [&]{ reader->readFile("a.txt", /*throw_on_not_found=*/true); }); - } - { - /// Reading with the right password is successful. - reader->setPassword("Qwe123"); - auto in = reader->readFile("a.txt", /*throw_on_not_found=*/true); - String str; - readStringUntilEOF(str, *in); - EXPECT_EQ(str, contents); + { + /// Try to read with a wrong password. + reader->setPassword("123Qwe"); + expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Wrong password", + [&]{ reader->readFile("a.txt", /*throw_on_not_found=*/true); }); + } + + { + /// Reading with the right password is successful. + reader->setPassword("Qwe123"); + auto in = reader->readFile("a.txt", /*throw_on_not_found=*/true); + String str; + readStringUntilEOF(str, *in); + EXPECT_EQ(str, contents); + } } } @@ -380,6 +392,53 @@ TEST_P(ArchiveReaderAndWriterTest, ArchiveNotExist) [&]{ createArchiveReader(getPathToArchive()); }); } + +TEST_P(ArchiveReaderAndWriterTest, LargeFile) +{ + /// Make an archive. + std::string_view contents = "The contents of a.txt\n"; + int times = 100000000; + { + auto writer = createArchiveWriter(getPathToArchive()); + { + auto out = writer->writeFile("a.txt", times * contents.size()); + for(int i = 0; i < times; i++) + { + writeString(contents, *out); + } + out->finalize(); + } + writer->finalize(); + } + + /// Read the archive. + auto reader = createArchiveReader(getPathToArchive()); + + ASSERT_TRUE(reader->fileExists("a.txt")); + + auto file_info = reader->getFileInfo("a.txt"); + EXPECT_EQ(file_info.uncompressed_size, contents.size() * times); + EXPECT_GT(file_info.compressed_size, 0); + + { + auto in = reader->readFile("a.txt", /*throw_on_not_found=*/true); + for(int i = 0; i < times; i++) + { + ASSERT_TRUE(checkString(String(contents), *in)); + } + } + + { + /// Use an enumerator. + auto enumerator = reader->firstFile(); + ASSERT_NE(enumerator, nullptr); + EXPECT_EQ(enumerator->getFileName(), "a.txt"); + EXPECT_EQ(enumerator->getFileInfo().uncompressed_size, contents.size() * times); + EXPECT_GT(enumerator->getFileInfo().compressed_size, 0); + EXPECT_FALSE(enumerator->nextFile()); + } +} + TEST(TarArchiveReaderTest, FileExists) { String archive_path = "archive.tar"; String filename = "file.txt"; @@ -508,7 +567,8 @@ namespace { const char * supported_archive_file_exts[] = { - ".zip" + ".zip", + ".tar" }; } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 06560bda2b6..ba69f7bda0d 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -591,6 +591,57 @@ def test_zip_archive_with_bad_compression_method(): ) +def test_tar_archive(): + backup_name = f"Disk('backups', 'archive.tar')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + assert os.path.isfile(get_path_to_backup(backup_name)) + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + +def test_tar_archive_with_password(): + backup_name = f"Disk('backups', 'archive_with_password.tar')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + expected_error = "Setting a password is not currently supported for tar archives" + assert expected_error in instance.query_and_get_error( + f"BACKUP TABLE test.table TO {backup_name} SETTINGS id='tar_archive_with_password', password='password123'" + ) + assert ( + instance.query( + "SELECT status FROM system.backups WHERE id='tar_archive_with_password'" + ) + == "BACKUP_FAILED\n" + ) + + +def test_tar_archive_with_bad_compression_method(): + backup_name = f"Disk('backups', 'archive_with_bad_compression_method.tar')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + expected_error = "Tar archives are currenly supported without compression" + assert expected_error in instance.query_and_get_error( + f"BACKUP TABLE test.table TO {backup_name} SETTINGS id='tar_archive_with_bad_compression_method', compression_method='foobar'" + ) + assert ( + instance.query( + "SELECT status FROM system.backups WHERE id='tar_archive_with_bad_compression_method'" + ) + == "BACKUP_FAILED\n" + ) + def test_async(): create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 634f14621c0..f30db01550f 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -453,6 +453,11 @@ def test_backup_to_zip(): backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.zip', 'minio', 'minio123')" check_backup_and_restore(storage_policy, backup_destination) +def test_backup_to_tar(): + storage_policy = "default" + backup_name = new_backup_name() + backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.tar', 'minio', 'minio123')" + check_backup_and_restore(storage_policy, backup_destination) def test_user_specific_auth(start_cluster): def create_user(user): From 1eb4cdf0f8df9b6d2ad210c9fb5a089290a4f242 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 5 Feb 2024 10:55:16 -0800 Subject: [PATCH 25/60] Add unit test coverage for ArchiveReaders --- src/IO/Archives/LibArchiveReader.cpp | 4 +--- src/IO/tests/gtest_archive_reader_and_writer.cpp | 8 ++++++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 83f34f31568..3b8b2c2a2a8 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -297,9 +297,7 @@ public: { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); } - - off_t getPosition() override { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); } - + String getFileName() const override { return handle.getFileName(); } size_t getFileSize() override { return handle.getFileInfo().uncompressed_size; } diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index e5909ce5869..121c1d84b2e 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -219,6 +219,10 @@ TEST_P(ArchiveReaderAndWriterTest, TwoFilesInArchive) ASSERT_TRUE(reader->fileExists("a.txt")); ASSERT_TRUE(reader->fileExists("b/c.txt")); + // Get all files + auto files = reader->getAllFiles(); + EXPECT_EQ(files.size(), 2); + EXPECT_EQ(reader->getFileInfo("a.txt").uncompressed_size, a_contents.size()); EXPECT_EQ(reader->getFileInfo("b/c.txt").uncompressed_size, c_contents.size()); @@ -274,6 +278,10 @@ TEST_P(ArchiveReaderAndWriterTest, TwoFilesInArchive) enumerator = reader->nextFile(std::move(in)); EXPECT_EQ(enumerator, nullptr); } + + // Get all files one last time + files = reader->getAllFiles(); + EXPECT_EQ(files.size(), 2); } From f9c0c99f3955d77cf7145ab9d07de475132bc6ca Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 5 Feb 2024 13:13:19 -0800 Subject: [PATCH 26/60] Fix issue with LibArchiveWriter setCompression method --- src/IO/Archives/LibArchiveWriter.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index c9a34cd014b..9158c226912 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -244,7 +244,8 @@ void LibArchiveWriter::finalize() void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) { - if (compression_method_.size() == 0 and compression_level == 1) + // throw an error unless setCompression is passed the defualt value + if (compression_method_.size() == 0 and compression_level == -1) { return; } From 3c77237dc4fa813145cb322a03acb12df32cc480 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 6 Feb 2024 10:13:14 -0800 Subject: [PATCH 27/60] getAllFiles now uses correct StreamInfo --- src/IO/Archives/LibArchiveReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 3b8b2c2a2a8..e9b759c40a9 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -119,7 +119,7 @@ public: std::unique_ptr rs; if(archive_read_function) { - read_stream = std::make_unique(archive_read_function()); + rs = std::make_unique(archive_read_function()); archive = openWithReader(&(*rs)); } else From e1fc5c8e484627ff745fc9dae9dc800663d6282b Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 7 Feb 2024 08:25:03 -0800 Subject: [PATCH 28/60] Add archive unit tests --- .../tests/gtest_archive_reader_and_writer.cpp | 107 +++++++++++++++++- 1 file changed, 102 insertions(+), 5 deletions(-) diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index 121c1d84b2e..dae1a2d5f9d 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -17,6 +17,8 @@ #include #include #include +#include + namespace DB::ErrorCodes @@ -344,6 +346,54 @@ TEST_P(ArchiveReaderAndWriterTest, InMemory) } +TEST_P(ArchiveReaderAndWriterTest, ManyFilesInMemory) +{ + String archive_in_memory; + int files = 1000; + size_t times = 1; + /// Make an archive. + { + auto writer = createArchiveWriter(getPathToArchive(), std::make_unique(archive_in_memory)); + { + for(int i = 0; i < files; i++) + { + auto filename = std::format("{}.txt", i); + auto contents = std::format("The contents of {}.txt", i); + auto out = writer->writeFile(filename, times * contents.size()); + for(int j = 0; j < times; j++) + { + writeString(contents, *out); + } + out->finalize(); + } + } + writer->finalize(); + } + + /// The created archive is really in memory. + ASSERT_FALSE(fs::exists(getPathToArchive())); + + /// Read the archive. + auto read_archive_func = [&]() -> std::unique_ptr { return std::make_unique(archive_in_memory); }; + auto reader = createArchiveReader(getPathToArchive(), read_archive_func, archive_in_memory.size()); + + for(int i = 0; i < files; i++) + { + auto filename = std::format("{}.txt", i); + auto contents = std::format("The contents of {}.txt", i); + ASSERT_TRUE(reader->fileExists(filename)); + EXPECT_EQ(reader->getFileInfo(filename).uncompressed_size, times * contents.size()); + + { + auto in = reader->readFile(filename, /*throw_on_not_found=*/true); + for(int j = 0; j < times; j++) + { + ASSERT_TRUE(checkString(String(contents), *in)); + } + } + } +} + TEST_P(ArchiveReaderAndWriterTest, Password) { auto writer = createArchiveWriter(getPathToArchive()); @@ -401,11 +451,57 @@ TEST_P(ArchiveReaderAndWriterTest, ArchiveNotExist) } +TEST_P(ArchiveReaderAndWriterTest, ManyFilesOnDisk) +{ + int files = 1000; + size_t times = 1; + /// Make an archive. + { + auto writer = createArchiveWriter(getPathToArchive()); + { + for(int i = 0; i < files; i++) + { + auto filename = std::format("{}.txt", i); + auto contents = std::format("The contents of {}.txt", i); + auto out = writer->writeFile(filename, times * contents.size()); + for(int j = 0; j < times; j++) + { + writeString(contents, *out); + } + out->finalize(); + } + } + writer->finalize(); + } + + /// The created archive is really in memory. + ASSERT_TRUE(fs::exists(getPathToArchive())); + + /// Read the archive. + auto reader = createArchiveReader(getPathToArchive()); + + for(int i = 0; i < files; i++) + { + auto filename = std::format("{}.txt", i); + auto contents = std::format("The contents of {}.txt", i); + ASSERT_TRUE(reader->fileExists(filename)); + EXPECT_EQ(reader->getFileInfo(filename).uncompressed_size, times * contents.size()); + + { + auto in = reader->readFile(filename, /*throw_on_not_found=*/true); + for(int j = 0; j < times; j++) + { + ASSERT_TRUE(checkString(String(contents), *in)); + } + } + } +} + TEST_P(ArchiveReaderAndWriterTest, LargeFile) { /// Make an archive. std::string_view contents = "The contents of a.txt\n"; - int times = 100000000; + int times = 10000000; { auto writer = createArchiveWriter(getPathToArchive()); { @@ -569,17 +665,18 @@ TEST(SevenZipArchiveReaderTest, ReadTwoFiles) { } -#if USE_MINIZIP namespace { const char * supported_archive_file_exts[] = { + #if USE_MINIZIP ".zip", - ".tar" + #endif + #if USE_LIBARCHIVE + ".tar", + #endif }; } INSTANTIATE_TEST_SUITE_P(All, ArchiveReaderAndWriterTest, ::testing::ValuesIn(supported_archive_file_exts)); - -#endif From ac6d74334c362c6ccef32bf0742ae11761132491 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Fri, 9 Feb 2024 14:15:53 -0800 Subject: [PATCH 29/60] Address review comments --- src/IO/Archives/IArchiveWriter.h | 2 +- src/IO/Archives/LibArchiveReader.cpp | 94 +++++++++++++++------------- src/IO/Archives/LibArchiveReader.h | 1 + src/IO/Archives/LibArchiveWriter.cpp | 14 ++--- src/IO/Archives/LibArchiveWriter.h | 2 +- src/IO/Archives/ZipArchiveWriter.cpp | 2 +- src/IO/Archives/ZipArchiveWriter.h | 2 +- 7 files changed, 61 insertions(+), 56 deletions(-) diff --git a/src/IO/Archives/IArchiveWriter.h b/src/IO/Archives/IArchiveWriter.h index 3dce83629fb..671bce28077 100644 --- a/src/IO/Archives/IArchiveWriter.h +++ b/src/IO/Archives/IArchiveWriter.h @@ -22,7 +22,7 @@ public: /// of the function `writeFile()` should be destroyed before next call of `writeFile()`. virtual std::unique_ptr writeFile(const String & filename) = 0; - virtual std::unique_ptr writeFile(const String & filename, const size_t & size) = 0; + virtual std::unique_ptr writeFile(const String & filename, size_t size) = 0; /// Returns true if there is an active instance of WriteBuffer returned by writeFile(). diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index e9b759c40a9..38e2ac8f1b5 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -2,7 +2,6 @@ #include #include #include - #include @@ -42,22 +41,22 @@ class LibArchiveReader::Handle { public: explicit Handle(std::string path_to_archive_, bool lock_on_reading_) - : path_to_archive(path_to_archive_), lock_on_reading(lock_on_reading_) + : path_to_archive(std::move(path_to_archive_)), lock_on_reading(lock_on_reading_) { current_archive = openWithPath(path_to_archive); } explicit Handle(std::string path_to_archive_, bool lock_on_reading_, const ReadArchiveFunction & archive_read_function_) - : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), lock_on_reading(lock_on_reading_) + : path_to_archive(std::move(path_to_archive_)), archive_read_function(archive_read_function_), lock_on_reading(lock_on_reading_) { read_stream = std::make_unique(archive_read_function()); - current_archive = openWithReader(&(*read_stream)); + current_archive = openWithReader(read_stream.get()); } Handle(const Handle &) = delete; Handle(Handle && other) noexcept - : current_archive(other.current_archive) + : read_stream(std::move(other.read_stream)) + , current_archive(other.current_archive) , current_entry(other.current_entry) - , read_stream(std::move(other.read_stream)) , archive_read_function(std::move(other.archive_read_function)) , lock_on_reading(other.lock_on_reading) @@ -115,17 +114,8 @@ public: std::vector getAllFiles(NameFilter filter) { - struct archive * archive; - std::unique_ptr rs; - if(archive_read_function) - { - rs = std::make_unique(archive_read_function()); - archive = openWithReader(&(*rs)); - } - else - { - archive = openWithPath(path_to_archive); - } + std::unique_ptr rs = archive_read_function ? std::make_unique(archive_read_function()) : nullptr; + auto archive = rs ? openWithReader(rs.get()) : openWithPath(path_to_archive); SCOPE_EXIT( close(archive); @@ -134,7 +124,7 @@ public: struct archive_entry * entry = nullptr; std::vector files; - int error = readNextHeader(current_archive, &entry); + int error = readNextHeader(archive, &entry); while (error == ARCHIVE_OK || error == ARCHIVE_RETRY) { chassert(entry != nullptr); @@ -176,11 +166,20 @@ public: return *file_info; } - struct archive * current_archive; - struct archive_entry * current_entry = nullptr; - bool valid = true; + la_ssize_t readData(void * buf, size_t len) + { + return archive_read_data(current_archive, buf, len); + } + + const char * getArchiveError() + { + return archive_error_string(current_archive); + } private: + using Archive = struct archive *; + using Entry = struct archive_entry *; + void checkError(int error) const { if (error == ARCHIVE_FATAL) @@ -193,16 +192,7 @@ private: file_info.reset(); } - static struct archive * openWithReader(StreamInfo * read_stream_) - { - auto * a = archive_read_new(); - archive_read_support_filter_all(a); - archive_read_support_format_all(a); - archive_read_open(a, read_stream_, nullptr, StreamInfo::read, nullptr); - return a; - } - - static struct archive * openWithPath(const String & path_to_archive) + Archive openWithReader(StreamInfo * read_stream_) { auto * archive = archive_read_new(); try @@ -230,7 +220,26 @@ private: return archive; } - static void close(struct archive * archive) + Archive openWithPath(const String & path_to_archive_) + { + auto * archive = archive_read_new(); + try + { + archive_read_support_filter_all(archive); + archive_read_support_format_all(archive); + if (archive_read_open_filename(archive, path_to_archive_.c_str(), 10240) != ARCHIVE_OK) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open archive {}: {}", quoteString(path_to_archive), archive_error_string(archive)); + } + catch (...) + { + close(archive); + throw; + } + + return archive; + } + + static void close(Archive archive) { if (archive) { @@ -248,9 +257,12 @@ private: return archive_read_next_header(archive, entry); } - const String path_to_archive; - std::unique_ptr read_stream = nullptr; - const IArchiveReader::ReadArchiveFunction archive_read_function; + String path_to_archive; + std::unique_ptr read_stream; + Archive current_archive; + Entry current_entry = nullptr; + bool valid = true; + IArchiveReader::ReadArchiveFunction archive_read_function; /// for some archive types when we are reading headers static variables are used /// which are not thread-safe @@ -307,10 +319,9 @@ public: private: bool nextImpl() override { - auto bytes_read = archive_read_data(handle.current_archive, internal_buffer.begin(), static_cast(internal_buffer.size())); - + auto bytes_read = handle.readData(internal_buffer.begin(), internal_buffer.size()); if (bytes_read < 0) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to read file {} from {}: {}", handle.getFileName(), path_to_archive, archive_error_string(handle.current_archive)); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to read file {} from {}: {}", handle.getFileName(), path_to_archive, handle.getArchiveError()); if (!bytes_read) return false; @@ -440,17 +451,14 @@ std::vector LibArchiveReader::getAllFiles(NameFilter filter) void LibArchiveReader::setPassword([[maybe_unused]] const String & password_) { - if (password_ != "") + if (password_.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to {} archive", archive_name); } LibArchiveReader::Handle LibArchiveReader::acquireHandle() { std::lock_guard lock{mutex}; - if (archive_read_function) - return Handle{path_to_archive, lock_on_reading, archive_read_function}; - else - return Handle{path_to_archive, lock_on_reading}; + return archive_read_function ? Handle{path_to_archive, lock_on_reading, archive_read_function} : Handle{path_to_archive, lock_on_reading}; } #endif diff --git a/src/IO/Archives/LibArchiveReader.h b/src/IO/Archives/LibArchiveReader.h index cca72e89d4d..83d4bb83eab 100644 --- a/src/IO/Archives/LibArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -3,6 +3,7 @@ #include "config.h" #include #include +#include namespace DB diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 9158c226912..1b0a70d1ede 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -176,7 +176,7 @@ LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_, std::unique_ if (archive_write_buffer_) { stream_info = std::make_unique(std::move(archive_write_buffer_)); - archive_write_open2(a, &(*stream_info), nullptr, &StreamInfo::memory_write, nullptr, nullptr); + archive_write_open2(a, stream_info.get(), nullptr, &StreamInfo::memory_write, nullptr, nullptr); } else { @@ -187,17 +187,13 @@ LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_, std::unique_ LibArchiveWriter::~LibArchiveWriter() { - if (!finalized) - { - if (!std::uncaught_exceptions() && std::current_exception() == nullptr) + if (!finalized && !std::uncaught_exceptions() && !std::current_exception()) chassert(false && "TarArchiveWriter is not finalized in destructor."); - } - if (a) archive_write_free(a); } -std::unique_ptr LibArchiveWriter::writeFile(const String & filename, const size_t & size) +std::unique_ptr LibArchiveWriter::writeFile(const String & filename, size_t size) { return std::make_unique(std::static_pointer_cast(shared_from_this()), filename, size); } @@ -245,7 +241,7 @@ void LibArchiveWriter::finalize() void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) { // throw an error unless setCompression is passed the defualt value - if (compression_method_.size() == 0 and compression_level == -1) + if (compression_method_.empty() == 0 && compression_level == -1) { return; } @@ -254,7 +250,7 @@ void LibArchiveWriter::setCompression(const String & compression_method_, int co void LibArchiveWriter::setPassword([[maybe_unused]] const String & password_) { - if (password_ == "") + if (password_.empty()) return; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for tar archives"); } diff --git a/src/IO/Archives/LibArchiveWriter.h b/src/IO/Archives/LibArchiveWriter.h index 982265cb2f1..6415009a512 100644 --- a/src/IO/Archives/LibArchiveWriter.h +++ b/src/IO/Archives/LibArchiveWriter.h @@ -35,7 +35,7 @@ public: /// passed in the the archive writer tries to infer the size by looking at the available /// data in the buffer, if next is called before all data is written to the buffer /// an exception is thrown. - std::unique_ptr writeFile(const String & filename, const size_t & size) override; + std::unique_ptr writeFile(const String & filename, size_t size) override; /// Returns true if there is an active instance of WriteBuffer returned by writeFile(). diff --git a/src/IO/Archives/ZipArchiveWriter.cpp b/src/IO/Archives/ZipArchiveWriter.cpp index 521e1e4e74c..0cfe921f33f 100644 --- a/src/IO/Archives/ZipArchiveWriter.cpp +++ b/src/IO/Archives/ZipArchiveWriter.cpp @@ -274,7 +274,7 @@ std::unique_ptr ZipArchiveWriter::writeFile(const Strin return std::make_unique(std::static_pointer_cast(shared_from_this()), filename); } -std::unique_ptr ZipArchiveWriter::writeFile(const String & filename, [[maybe_unused]] const size_t & size) +std::unique_ptr ZipArchiveWriter::writeFile(const String & filename, [[maybe_unused]] size_t size) { return ZipArchiveWriter::writeFile(filename); } diff --git a/src/IO/Archives/ZipArchiveWriter.h b/src/IO/Archives/ZipArchiveWriter.h index 0b8260b1f2f..b2b77dce7e1 100644 --- a/src/IO/Archives/ZipArchiveWriter.h +++ b/src/IO/Archives/ZipArchiveWriter.h @@ -32,7 +32,7 @@ public: /// of the function `writeFile()` should be destroyed before next call of `writeFile()`. std::unique_ptr writeFile(const String & filename) override; - std::unique_ptr writeFile(const String & filename, const size_t & size) override; + std::unique_ptr writeFile(const String & filename, size_t size) override; /// Returns true if there is an active instance of WriteBuffer returned by writeFile(). From 4c1ac01e25e83b1fa2bd9a52be725fcfd6b86043 Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Fri, 9 Feb 2024 17:30:26 -0500 Subject: [PATCH 30/60] Update src/IO/Archives/LibArchiveReader.cpp Co-authored-by: ltrk2 <107155950+ltrk2@users.noreply.github.com> --- src/IO/Archives/LibArchiveReader.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 38e2ac8f1b5..7df79a94e1d 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -309,7 +309,6 @@ public: { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); } - String getFileName() const override { return handle.getFileName(); } size_t getFileSize() override { return handle.getFileInfo().uncompressed_size; } From 48ad506fe3747451bb3cacfc94d40f8a3589a0f5 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 12 Feb 2024 13:33:35 -0800 Subject: [PATCH 31/60] Cleaned up the uses of the libarchive structs --- src/IO/Archives/LibArchiveReader.cpp | 4 ++-- src/IO/Archives/LibArchiveWriter.cpp | 30 ++++++++++++++-------------- src/IO/Archives/LibArchiveWriter.h | 7 +++++-- 3 files changed, 22 insertions(+), 19 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 7df79a94e1d..df13e96c669 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -121,7 +121,7 @@ public: close(archive); ); - struct archive_entry * entry = nullptr; + Entry entry = nullptr; std::vector files; int error = readNextHeader(archive, &entry); @@ -132,7 +132,7 @@ public: if (!filter || filter(name)) files.push_back(std::move(name)); - error = readNextHeader(current_archive, &entry); + error = readNextHeader(archive, &entry); } checkError(error); diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 1b0a70d1ede..ecb06b9c1df 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -57,7 +57,7 @@ public: : WriteBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0), archive_writer(archive_writer_), filename(filename_), size(size_) { startWritingFile(); - a = archive_writer_->getArchive(); + archive = archive_writer_->getArchive(); entry = nullptr; } @@ -94,7 +94,7 @@ private: if (entry == nullptr) writeEntry(); ssize_t to_write = offset(); - ssize_t written = archive_write_data(a, working_buffer.begin(), offset()); + ssize_t written = archive_write_data(archive, working_buffer.begin(), offset()); if (written != to_write) { throw Exception( @@ -115,7 +115,7 @@ private: archive_entry_set_size(entry, expected_size); archive_entry_set_filetype(entry, static_cast<__LA_MODE_T>(0100000)); archive_entry_set_perm(entry, 0644); - checkResult(archive_write_header(a, entry)); + checkResult(archive_write_header(archive, entry)); } size_t getSize() const @@ -155,8 +155,8 @@ private: std::weak_ptr archive_writer; const String filename; - struct archive_entry * entry; - struct archive * a; + Entry entry; + Archive archive; size_t size; size_t expected_size; }; @@ -169,18 +169,18 @@ LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_) : path_to_ar LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_) : path_to_archive(path_to_archive_) { - a = archive_write_new(); - archive_write_set_format_pax_restricted(a); + archive = archive_write_new(); + archive_write_set_format_pax_restricted(archive); //this allows use to write directly to a writer buffer rather than an intermediate buffer in LibArchive //archive_write_set_bytes_per_block(a, 0); if (archive_write_buffer_) { stream_info = std::make_unique(std::move(archive_write_buffer_)); - archive_write_open2(a, stream_info.get(), nullptr, &StreamInfo::memory_write, nullptr, nullptr); + archive_write_open2(archive, stream_info.get(), nullptr, &StreamInfo::memory_write, nullptr, nullptr); } else { - archive_write_open_filename(a, path_to_archive.c_str()); + archive_write_open_filename(archive, path_to_archive.c_str()); } } @@ -189,8 +189,8 @@ LibArchiveWriter::~LibArchiveWriter() { if (!finalized && !std::uncaught_exceptions() && !std::current_exception()) chassert(false && "TarArchiveWriter is not finalized in destructor."); - if (a) - archive_write_free(a); + if (archive) + archive_write_free(archive); } std::unique_ptr LibArchiveWriter::writeFile(const String & filename, size_t size) @@ -228,8 +228,8 @@ void LibArchiveWriter::finalize() std::lock_guard lock{mutex}; if (finalized) return; - if (a) - archive_write_close(a); + if (archive) + archive_write_close(archive); if (stream_info) { stream_info->archive_write_buffer->finalize(); @@ -255,10 +255,10 @@ void LibArchiveWriter::setPassword([[maybe_unused]] const String & password_) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for tar archives"); } -struct archive * LibArchiveWriter::getArchive() +LibArchiveWriter::Archive LibArchiveWriter::getArchive() { std::lock_guard lock{mutex}; - return a; + return archive; } } #endif diff --git a/src/IO/Archives/LibArchiveWriter.h b/src/IO/Archives/LibArchiveWriter.h index 6415009a512..a69ccfa9d09 100644 --- a/src/IO/Archives/LibArchiveWriter.h +++ b/src/IO/Archives/LibArchiveWriter.h @@ -59,13 +59,16 @@ private: class WriteBufferFromLibArchive; class StreamInfo; - struct archive * getArchive(); + using Archive = struct archive *; + using Entry = struct archive_entry *; + + Archive getArchive(); void startWritingFile(); void endWritingFile(); String path_to_archive; std::unique_ptr stream_info TSA_GUARDED_BY(mutex) = nullptr; - struct archive * a TSA_GUARDED_BY(mutex) = nullptr; + Archive archive TSA_GUARDED_BY(mutex) = nullptr; bool is_writing_file TSA_GUARDED_BY(mutex) = false; bool finalized TSA_GUARDED_BY(mutex) = false; mutable std::mutex mutex; From c53021f55ecbb1be0c9c30c2413c87fc332ecb68 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 14 Feb 2024 05:31:26 -0800 Subject: [PATCH 32/60] Fix formating --- src/IO/Archives/LibArchiveReader.cpp | 62 +++++++++++-------- src/IO/Archives/LibArchiveWriter.cpp | 24 +++---- .../test_backup_restore_s3/test.py | 1 + 3 files changed, 49 insertions(+), 38 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index df13e96c669..895714cadfa 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -1,8 +1,8 @@ +#include #include #include #include #include -#include namespace DB @@ -21,10 +21,7 @@ extern const int UNSUPPORTED_METHOD; class LibArchiveReader::StreamInfo { public: - explicit StreamInfo(std::unique_ptr read_buffer_) - : read_buffer(std::move(read_buffer_)) - { - } + explicit StreamInfo(std::unique_ptr read_buffer_) : read_buffer(std::move(read_buffer_)) { } static ssize_t read([[maybe_unused]] struct archive * a, void * client_data, const void ** buff) { @@ -41,12 +38,12 @@ class LibArchiveReader::Handle { public: explicit Handle(std::string path_to_archive_, bool lock_on_reading_) - : path_to_archive(std::move(path_to_archive_)), lock_on_reading(lock_on_reading_) + : path_to_archive(std::move(path_to_archive_)), lock_on_reading(lock_on_reading_) { current_archive = openWithPath(path_to_archive); } explicit Handle(std::string path_to_archive_, bool lock_on_reading_, const ReadArchiveFunction & archive_read_function_) - : path_to_archive(std::move(path_to_archive_)), archive_read_function(archive_read_function_), lock_on_reading(lock_on_reading_) + : path_to_archive(std::move(path_to_archive_)), archive_read_function(archive_read_function_), lock_on_reading(lock_on_reading_) { read_stream = std::make_unique(archive_read_function()); current_archive = openWithReader(read_stream.get()); @@ -59,7 +56,7 @@ public: , current_entry(other.current_entry) , archive_read_function(std::move(other.archive_read_function)) , lock_on_reading(other.lock_on_reading) - + { other.current_archive = nullptr; other.current_entry = nullptr; @@ -114,12 +111,11 @@ public: std::vector getAllFiles(NameFilter filter) { - std::unique_ptr rs = archive_read_function ? std::make_unique(archive_read_function()) : nullptr; + std::unique_ptr rs + = archive_read_function ? std::make_unique(archive_read_function()) : nullptr; auto archive = rs ? openWithReader(rs.get()) : openWithPath(path_to_archive); - SCOPE_EXIT( - close(archive); - ); + SCOPE_EXIT(close(archive);); Entry entry = nullptr; @@ -166,15 +162,9 @@ public: return *file_info; } - la_ssize_t readData(void * buf, size_t len) - { - return archive_read_data(current_archive, buf, len); - } + la_ssize_t readData(void * buf, size_t len) { return archive_read_data(current_archive, buf, len); } - const char * getArchiveError() - { - return archive_error_string(current_archive); - } + const char * getArchiveError() { return archive_error_string(current_archive); } private: using Archive = struct archive *; @@ -183,7 +173,10 @@ private: void checkError(int error) const { if (error == ARCHIVE_FATAL) - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Failed to read archive while fetching all files: {}", archive_error_string(current_archive)); + throw Exception( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, + "Failed to read archive while fetching all files: {}", + archive_error_string(current_archive)); } void resetFileInfo() @@ -228,7 +221,11 @@ private: archive_read_support_filter_all(archive); archive_read_support_format_all(archive); if (archive_read_open_filename(archive, path_to_archive_.c_str(), 10240) != ARCHIVE_OK) - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open archive {}: {}", quoteString(path_to_archive), archive_error_string(archive)); + throw Exception( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, + "Couldn't open archive {}: {}", + quoteString(path_to_archive), + archive_error_string(archive)); } catch (...) { @@ -320,7 +317,12 @@ private: { auto bytes_read = handle.readData(internal_buffer.begin(), internal_buffer.size()); if (bytes_read < 0) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to read file {} from {}: {}", handle.getFileName(), path_to_archive, handle.getArchiveError()); + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Failed to read file {} from {}: {}", + handle.getFileName(), + path_to_archive, + handle.getArchiveError()); if (!bytes_read) return false; @@ -368,7 +370,11 @@ LibArchiveReader::FileInfo LibArchiveReader::getFileInfo(const String & filename { Handle handle = acquireHandle(); if (!handle.locateFile(filename)) - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: File {} was not found in archive", path_to_archive, quoteString(filename)); + throw Exception( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, + "Couldn't unpack archive {}: File {} was not found in archive", + path_to_archive, + quoteString(filename)); return handle.getFileInfo(); } @@ -388,7 +394,10 @@ std::unique_ptr LibArchiveReader::readFile(const String { if (throw_on_not_found) throw Exception( - ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: File {} was not found in archive", path_to_archive, quoteString(filename)); + ErrorCodes::CANNOT_UNPACK_ARCHIVE, + "Couldn't unpack archive {}: File {} was not found in archive", + path_to_archive, + quoteString(filename)); return nullptr; } return std::make_unique(std::move(handle), path_to_archive); @@ -457,7 +466,8 @@ void LibArchiveReader::setPassword([[maybe_unused]] const String & password_) LibArchiveReader::Handle LibArchiveReader::acquireHandle() { std::lock_guard lock{mutex}; - return archive_read_function ? Handle{path_to_archive, lock_on_reading, archive_read_function} : Handle{path_to_archive, lock_on_reading}; + return archive_read_function ? Handle{path_to_archive, lock_on_reading, archive_read_function} + : Handle{path_to_archive, lock_on_reading}; } #endif diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index ecb06b9c1df..9592135fc95 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -9,16 +9,13 @@ #if USE_LIBARCHIVE -// this implemation follows the ZipArchiveWriter implemation as closely as possible. +// this implemation follows the ZipArchiveWriter implemation as closely as possible. namespace DB { namespace ErrorCodes { extern const int CANNOT_PACK_ARCHIVE; -extern const int LOGICAL_ERROR; -extern const int CANNOT_READ_ALL_DATA; -extern const int UNSUPPORTED_METHOD; extern const int NOT_IMPLEMENTED; } @@ -135,8 +132,13 @@ private: } if (throw_if_error and bytes != expected_size) { - throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't pack tar archive: Wrote {} of expected {} , filename={}", bytes, expected_size, quoteString(filename)); - } + throw Exception( + ErrorCodes::CANNOT_PACK_ARCHIVE, + "Couldn't pack tar archive: Wrote {} of expected {} , filename={}", + bytes, + expected_size, + quoteString(filename)); + } } void endWritingFile() @@ -188,7 +190,7 @@ LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_, std::unique_ LibArchiveWriter::~LibArchiveWriter() { if (!finalized && !std::uncaught_exceptions() && !std::current_exception()) - chassert(false && "TarArchiveWriter is not finalized in destructor."); + chassert(false && "TarArchiveWriter is not finalized in destructor."); if (archive) archive_write_free(archive); } @@ -240,12 +242,10 @@ void LibArchiveWriter::finalize() void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) { - // throw an error unless setCompression is passed the defualt value + // throw an error unless setCompression is passed the default value if (compression_method_.empty() == 0 && compression_level == -1) - { - return; - } - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Tar archives are currenly supported without compression"); + return; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "tar archives are currently supported without compression"); } void LibArchiveWriter::setPassword([[maybe_unused]] const String & password_) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index f30db01550f..00ed5ee4d13 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -453,6 +453,7 @@ def test_backup_to_zip(): backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.zip', 'minio', 'minio123')" check_backup_and_restore(storage_policy, backup_destination) + def test_backup_to_tar(): storage_policy = "default" backup_name = new_backup_name() From dc74cebe4e9af57a526e0ff0a718b6582d047efe Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 14 Feb 2024 05:49:00 -0800 Subject: [PATCH 33/60] Fix formating --- src/IO/Archives/LibArchiveReader.h | 21 ++++++++++++------- .../test_backup_restore_new/test.py | 1 + 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.h b/src/IO/Archives/LibArchiveReader.h index 83d4bb83eab..148d5dd17f2 100644 --- a/src/IO/Archives/LibArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -1,9 +1,9 @@ #pragma once -#include "config.h" #include #include #include +#include "config.h" namespace DB @@ -53,7 +53,8 @@ protected: /// Constructs an archive's reader that will read from a file in the local filesystem. LibArchiveReader(std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_); - LibArchiveReader(std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_, const ReadArchiveFunction & archive_read_function_); + LibArchiveReader( + std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_, const ReadArchiveFunction & archive_read_function_); private: class ReadBufferFromLibArchive; @@ -68,22 +69,28 @@ private: const String path_to_archive; const ReadArchiveFunction archive_read_function; mutable std::mutex mutex; - }; class TarArchiveReader : public LibArchiveReader { public: - explicit TarArchiveReader(std::string path_to_archive) : LibArchiveReader("tar", /*lock_on_reading_=*/ true, std::move(path_to_archive)) { } - - explicit TarArchiveReader(std::string path_to_archive, const ReadArchiveFunction & archive_read_function ) : LibArchiveReader("tar", /*lock_on_reading_=*/ true, std::move(path_to_archive), archive_read_function) { } + explicit TarArchiveReader(std::string path_to_archive) : LibArchiveReader("tar", /*lock_on_reading_=*/true, std::move(path_to_archive)) + { + } + explicit TarArchiveReader(std::string path_to_archive, const ReadArchiveFunction & archive_read_function) + : LibArchiveReader("tar", /*lock_on_reading_=*/true, std::move(path_to_archive), archive_read_function) + { + } }; class SevenZipArchiveReader : public LibArchiveReader { public: - explicit SevenZipArchiveReader(std::string path_to_archive) : LibArchiveReader("7z", /*lock_on_reading_=*/ false, std::move(path_to_archive)) { } + explicit SevenZipArchiveReader(std::string path_to_archive) + : LibArchiveReader("7z", /*lock_on_reading_=*/false, std::move(path_to_archive)) + { + } }; #endif diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index ba69f7bda0d..760f02f1a36 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -642,6 +642,7 @@ def test_tar_archive_with_bad_compression_method(): == "BACKUP_FAILED\n" ) + def test_async(): create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" From 1bb9115abd6941cd39d5a5aa33c561b4d19f2dc8 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 14 Feb 2024 06:18:58 -0800 Subject: [PATCH 34/60] Add documentation --- docs/en/operations/backup.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index b1f2135c476..e4933414647 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -168,6 +168,20 @@ RESTORE TABLE test.table PARTITIONS '2', '3' FROM Disk('backups', 'filename.zip') ``` +### Backups as tar archives + +Backups can also be stored as tar archives. The functionality is the same as for zip, except that compressing the archive or adding a password is not supported. + +Write a backup as a tar: +``` +BACKUP TABLE test.table TO Disk('backups', '1.tar') +``` + +Corresponding restore: +``` +RESTORE TABLE test.table FROM Disk('backups', '1.tar') +``` + ### Check the status of backups The backup command returns an `id` and `status`, and that `id` can be used to get the status of the backup. This is very useful to check the progress of long ASYNC backups. The example below shows a failure that happened when trying to overwrite an existing backup file: From d0776e5d2a81c15af2ab4b052ecad7c3b9c9b8a8 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 14 Feb 2024 15:42:00 -0800 Subject: [PATCH 35/60] Fixes tidy issue and error formating for test --- src/IO/Archives/LibArchiveReader.cpp | 2 +- src/IO/Archives/LibArchiveWriter.cpp | 2 +- tests/integration/test_backup_restore_new/test.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 895714cadfa..592358df004 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -113,7 +113,7 @@ public: { std::unique_ptr rs = archive_read_function ? std::make_unique(archive_read_function()) : nullptr; - auto archive = rs ? openWithReader(rs.get()) : openWithPath(path_to_archive); + auto * archive = rs ? openWithReader(rs.get()) : openWithPath(path_to_archive); SCOPE_EXIT(close(archive);); diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 9592135fc95..ab600e28a54 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -243,7 +243,7 @@ void LibArchiveWriter::finalize() void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) { // throw an error unless setCompression is passed the default value - if (compression_method_.empty() == 0 && compression_level == -1) + if (compression_method_.empty() && compression_level == -1) return; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "tar archives are currently supported without compression"); } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 760f02f1a36..a01bb9d7732 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -631,7 +631,7 @@ def test_tar_archive_with_bad_compression_method(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" - expected_error = "Tar archives are currenly supported without compression" + expected_error = "tar archives are currenly supported without compression" assert expected_error in instance.query_and_get_error( f"BACKUP TABLE test.table TO {backup_name} SETTINGS id='tar_archive_with_bad_compression_method', compression_method='foobar'" ) From ef7a66b542f7c459fe8a1c3aa2732ac05f7c2a5f Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 15 Feb 2024 05:36:15 -0800 Subject: [PATCH 36/60] Update error messages and error handling --- src/IO/Archives/LibArchiveReader.cpp | 3 ++- src/IO/Archives/LibArchiveWriter.cpp | 4 ++-- src/IO/Archives/LibArchiveWriter.h | 6 ++---- tests/integration/test_backup_restore_new/test.py | 2 +- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 592358df004..49322ce9be1 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -460,7 +460,8 @@ std::vector LibArchiveReader::getAllFiles(NameFilter filter) void LibArchiveReader::setPassword([[maybe_unused]] const String & password_) { if (password_.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to {} archive", archive_name); + return; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot set password to {} archive", archive_name); } LibArchiveReader::Handle LibArchiveReader::acquireHandle() diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index ab600e28a54..3411d94df1e 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -240,12 +240,12 @@ void LibArchiveWriter::finalize() finalized = true; } -void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) +void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) { // throw an error unless setCompression is passed the default value if (compression_method_.empty() && compression_level == -1) return; - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "tar archives are currently supported without compression"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Compressing tar archives is currently not supported"); } void LibArchiveWriter::setPassword([[maybe_unused]] const String & password_) diff --git a/src/IO/Archives/LibArchiveWriter.h b/src/IO/Archives/LibArchiveWriter.h index a69ccfa9d09..3c88fdbc30c 100644 --- a/src/IO/Archives/LibArchiveWriter.h +++ b/src/IO/Archives/LibArchiveWriter.h @@ -46,14 +46,12 @@ public: /// (Unless an error appeared and the archive is in fact no longer needed.) void finalize() override; - static constexpr const int kDefaultCompressionLevel = -1; - /// Sets compression method and level. /// Changing them will affect next file in the archive. - void setCompression(const String & /* compression_method */, int /* compression_level */ = kDefaultCompressionLevel) override; + void setCompression(const String & method, int level) override; /// Sets password. If the password is not empty it will enable encryption in the archive. - void setPassword(const String & /* password */) override; + void setPassword(const String & password) override; private: class WriteBufferFromLibArchive; diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index a01bb9d7732..70b328e525a 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -631,7 +631,7 @@ def test_tar_archive_with_bad_compression_method(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" - expected_error = "tar archives are currenly supported without compression" + expected_error = "Compressing tar archives is currently not supported" assert expected_error in instance.query_and_get_error( f"BACKUP TABLE test.table TO {backup_name} SETTINGS id='tar_archive_with_bad_compression_method', compression_method='foobar'" ) From 2e2b89f9aa31c00a7f18eb732bf92f3386a0c6f8 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 15 Feb 2024 06:01:04 -0800 Subject: [PATCH 37/60] Fix whitespace --- src/IO/Archives/LibArchiveWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 3411d94df1e..b870ecbc1b9 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -240,7 +240,7 @@ void LibArchiveWriter::finalize() finalized = true; } -void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) +void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) { // throw an error unless setCompression is passed the default value if (compression_method_.empty() && compression_level == -1) From 5fdeebf7857c73fbca8887cc20f975a5cfddc63b Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Thu, 15 Feb 2024 13:30:17 -0800 Subject: [PATCH 38/60] Fix parameter naming --- src/IO/Archives/LibArchiveWriter.cpp | 4 ++-- src/IO/Archives/LibArchiveWriter.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index b870ecbc1b9..17ff011ebed 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -240,10 +240,10 @@ void LibArchiveWriter::finalize() finalized = true; } -void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level) +void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level_) { // throw an error unless setCompression is passed the default value - if (compression_method_.empty() && compression_level == -1) + if (compression_method_.empty() && compression_level_ == -1) return; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Compressing tar archives is currently not supported"); } diff --git a/src/IO/Archives/LibArchiveWriter.h b/src/IO/Archives/LibArchiveWriter.h index 3c88fdbc30c..81aaf35c8b8 100644 --- a/src/IO/Archives/LibArchiveWriter.h +++ b/src/IO/Archives/LibArchiveWriter.h @@ -48,7 +48,7 @@ public: /// Sets compression method and level. /// Changing them will affect next file in the archive. - void setCompression(const String & method, int level) override; + void setCompression(const String & compression_method_, int compression_level_) override; /// Sets password. If the password is not empty it will enable encryption in the archive. void setPassword(const String & password) override; From e085b0fcf0ddb3c76c15b9ff4eaf8bb8b35e5ef1 Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Tue, 20 Feb 2024 07:49:55 -0500 Subject: [PATCH 39/60] Update src/IO/Archives/IArchiveWriter.h Co-authored-by: Antonio Andelic --- src/IO/Archives/IArchiveWriter.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/Archives/IArchiveWriter.h b/src/IO/Archives/IArchiveWriter.h index 671bce28077..c8f0c609da1 100644 --- a/src/IO/Archives/IArchiveWriter.h +++ b/src/IO/Archives/IArchiveWriter.h @@ -24,7 +24,6 @@ public: virtual std::unique_ptr writeFile(const String & filename, size_t size) = 0; - /// Returns true if there is an active instance of WriteBuffer returned by writeFile(). /// This function should be used mostly for debugging purposes. virtual bool isWritingFile() const = 0; From f94b5e95b2593abea452ae03b33db98f0059aea1 Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Tue, 20 Feb 2024 07:50:19 -0500 Subject: [PATCH 40/60] Update src/IO/Archives/LibArchiveReader.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveReader.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 49322ce9be1..7fa9b16b974 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -42,6 +42,7 @@ public: { current_archive = openWithPath(path_to_archive); } + explicit Handle(std::string path_to_archive_, bool lock_on_reading_, const ReadArchiveFunction & archive_read_function_) : path_to_archive(std::move(path_to_archive_)), archive_read_function(archive_read_function_), lock_on_reading(lock_on_reading_) { From 84814f2a0c7309fef29de7efe6159bf96130f801 Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Tue, 20 Feb 2024 07:51:26 -0500 Subject: [PATCH 41/60] Update src/IO/Archives/LibArchiveWriter.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 17ff011ebed..704c55c17d5 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -248,7 +248,7 @@ void LibArchiveWriter::setCompression(const String & compression_method_, int co throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Compressing tar archives is currently not supported"); } -void LibArchiveWriter::setPassword([[maybe_unused]] const String & password_) +void LibArchiveWriter::setPassword(const String & password_) { if (password_.empty()) return; From 714fdeb906464915d9c192cbc254a31c269717cd Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Tue, 20 Feb 2024 07:51:32 -0500 Subject: [PATCH 42/60] Update src/IO/Archives/LibArchiveWriter.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 704c55c17d5..c5c586fea41 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -123,7 +123,7 @@ private: return offset(); } - void closeFile([[maybe_unused]] bool throw_if_error) + void closeFile(bool throw_if_error) { if (entry) { From c3118ca742ec7900a33475f1e4d3d4eee5a1f335 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 20 Feb 2024 06:50:59 -0800 Subject: [PATCH 43/60] Address review comments --- src/IO/Archives/LibArchiveWriter.cpp | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index c5c586fea41..5c03bc6c762 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -37,7 +37,7 @@ public: explicit StreamInfo(std::unique_ptr archive_write_buffer_) : archive_write_buffer(std::move(archive_write_buffer_)) { } - static ssize_t memory_write([[maybe_unused]] struct archive * a, void * client_data, const void * buff, size_t length) + static ssize_t memory_write([[maybe_unused]] struct archive * archive, void * client_data, const void * buff, size_t length) { auto * stream_info = reinterpret_cast(client_data); stream_info->archive_write_buffer->write(reinterpret_cast(buff), length); @@ -58,7 +58,6 @@ public: entry = nullptr; } - ~WriteBufferFromLibArchive() override { try @@ -82,7 +81,6 @@ public: void sync() override { next(); } std::string getFileName() const override { return filename; } - private: void nextImpl() override { @@ -103,7 +101,6 @@ private: } } - void writeEntry() { expected_size = getSize(); @@ -163,11 +160,6 @@ private: size_t expected_size; }; -LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_) : path_to_archive(path_to_archive_) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not Implemented"); -} - LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_) : path_to_archive(path_to_archive_) { @@ -186,11 +178,9 @@ LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_, std::unique_ } } - LibArchiveWriter::~LibArchiveWriter() { - if (!finalized && !std::uncaught_exceptions() && !std::current_exception()) - chassert(false && "TarArchiveWriter is not finalized in destructor."); + chassert((finalized || std::uncaught_exceptions() || std::current_exception()) && "TarArchiveWriter is not finalized in destructor."); if (archive) archive_write_free(archive); } From 92b446a55231cae5b559c7315100b4e6fead6686 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 20 Feb 2024 15:34:02 -0800 Subject: [PATCH 44/60] Add support for tar archive compression --- src/IO/Archives/LibArchiveWriter.cpp | 28 ++++------- src/IO/Archives/LibArchiveWriter.h | 24 ++++----- src/IO/Archives/TarArchiveWriter.cpp | 37 ++++++++++++++ src/IO/Archives/TarArchiveWriter.h | 28 +++++++++++ src/IO/Archives/createArchiveReader.cpp | 20 +++----- src/IO/Archives/createArchiveWriter.cpp | 8 ++- .../hasRegisteredArchiveFileExtension.cpp | 3 +- .../tests/gtest_archive_reader_and_writer.cpp | 7 ++- .../test_backup_restore_new/test.py | 50 ++++++++++++++++++- .../test_backup_restore_s3/test.py | 22 ++++++++ 10 files changed, 178 insertions(+), 49 deletions(-) create mode 100644 src/IO/Archives/TarArchiveWriter.cpp create mode 100644 src/IO/Archives/TarArchiveWriter.h diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 5c03bc6c762..84aed5315b7 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -163,24 +163,26 @@ private: LibArchiveWriter::LibArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_) : path_to_archive(path_to_archive_) { + if (archive_write_buffer_) + stream_info = std::make_unique(std::move(archive_write_buffer_)); +} + +void LibArchiveWriter::createArchive() +{ + std::lock_guard lock{mutex}; archive = archive_write_new(); - archive_write_set_format_pax_restricted(archive); + setFormatAndSettings(archive); //this allows use to write directly to a writer buffer rather than an intermediate buffer in LibArchive //archive_write_set_bytes_per_block(a, 0); - if (archive_write_buffer_) - { - stream_info = std::make_unique(std::move(archive_write_buffer_)); + if (stream_info) archive_write_open2(archive, stream_info.get(), nullptr, &StreamInfo::memory_write, nullptr, nullptr); - } else - { archive_write_open_filename(archive, path_to_archive.c_str()); - } } LibArchiveWriter::~LibArchiveWriter() { - chassert((finalized || std::uncaught_exceptions() || std::current_exception()) && "TarArchiveWriter is not finalized in destructor."); + chassert((finalized || std::uncaught_exceptions() || std::current_exception()) && "LibArchiveWriter is not finalized in destructor."); if (archive) archive_write_free(archive); } @@ -230,19 +232,11 @@ void LibArchiveWriter::finalize() finalized = true; } -void LibArchiveWriter::setCompression(const String & compression_method_, int compression_level_) -{ - // throw an error unless setCompression is passed the default value - if (compression_method_.empty() && compression_level_ == -1) - return; - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Compressing tar archives is currently not supported"); -} - void LibArchiveWriter::setPassword(const String & password_) { if (password_.empty()) return; - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for tar archives"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for libarchive"); } LibArchiveWriter::Archive LibArchiveWriter::getArchive() diff --git a/src/IO/Archives/LibArchiveWriter.h b/src/IO/Archives/LibArchiveWriter.h index 81aaf35c8b8..d42db73b3e8 100644 --- a/src/IO/Archives/LibArchiveWriter.h +++ b/src/IO/Archives/LibArchiveWriter.h @@ -17,9 +17,6 @@ class WriteBufferFromFileBase; class LibArchiveWriter : public IArchiveWriter { public: - /// Constructs an archive that will be written as a file in the local filesystem. - [[noreturn]] explicit LibArchiveWriter(const String & path_to_archive_); - /// Constructs an archive that will be written as a file in the local filesystem. explicit LibArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_); @@ -37,7 +34,6 @@ public: /// an exception is thrown. std::unique_ptr writeFile(const String & filename, size_t size) override; - /// Returns true if there is an active instance of WriteBuffer returned by writeFile(). /// This function should be used mostly for debugging purposes. bool isWritingFile() const override; @@ -48,30 +44,34 @@ public: /// Sets compression method and level. /// Changing them will affect next file in the archive. - void setCompression(const String & compression_method_, int compression_level_) override; + //void setCompression(const String & compression_method_, int compression_level_) override; /// Sets password. If the password is not empty it will enable encryption in the archive. void setPassword(const String & password) override; +protected: + using Archive = struct archive *; + using Entry = struct archive_entry *; + + //derived classes must call createArcive. createArchive calls initArchive + void createArchive(); + virtual void setFormatAndSettings(Archive) = 0; + + Archive archive = nullptr; + String path_to_archive; + private: class WriteBufferFromLibArchive; class StreamInfo; - using Archive = struct archive *; - using Entry = struct archive_entry *; - Archive getArchive(); void startWritingFile(); void endWritingFile(); - String path_to_archive; std::unique_ptr stream_info TSA_GUARDED_BY(mutex) = nullptr; - Archive archive TSA_GUARDED_BY(mutex) = nullptr; bool is_writing_file TSA_GUARDED_BY(mutex) = false; bool finalized TSA_GUARDED_BY(mutex) = false; mutable std::mutex mutex; }; - } - #endif diff --git a/src/IO/Archives/TarArchiveWriter.cpp b/src/IO/Archives/TarArchiveWriter.cpp new file mode 100644 index 00000000000..7bed0fd32a4 --- /dev/null +++ b/src/IO/Archives/TarArchiveWriter.cpp @@ -0,0 +1,37 @@ +#include + +#if USE_LIBARCHIVE +namespace DB +{ +namespace ErrorCodes +{ +extern const int CANNOT_PACK_ARCHIVE; +extern const int NOT_IMPLEMENTED; +} +void TarArchiveWriter::setCompression(const String & compression_method_, int compression_level_) +{ + // throw an error unless setCompression is passed the default value + if (compression_method_.empty() && compression_level_ == -1) + return; + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Using compression_method and compression_level options are not supported for tar archives"); +} + +void TarArchiveWriter::setFormatAndSettings(Archive archive_) +{ + archive_write_set_format_pax_restricted(archive_); + inferCompressionFromPath(); +} + +void TarArchiveWriter::inferCompressionFromPath() +{ + if (path_to_archive.ends_with(".gz")) + archive_write_add_filter_gzip(archive); + else if (path_to_archive.ends_with(".bz2")) + archive_write_add_filter_bzip2(archive); + else if (path_to_archive.ends_with(".lzma")) + archive_write_add_filter_lzma(archive); + //else path ends in .tar and we dont do any compression +} +} +#endif diff --git a/src/IO/Archives/TarArchiveWriter.h b/src/IO/Archives/TarArchiveWriter.h new file mode 100644 index 00000000000..553f68202f2 --- /dev/null +++ b/src/IO/Archives/TarArchiveWriter.h @@ -0,0 +1,28 @@ +#pragma once + +#include "config.h" + +#if USE_LIBARCHIVE + +# include +namespace DB +{ +using namespace std::literals; + +class TarArchiveWriter : public LibArchiveWriter +{ +public: + static constexpr std::array TarExtensions{".tar"sv, ".tar.gz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; + + explicit TarArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_) + : LibArchiveWriter(path_to_archive_, std::move(archive_write_buffer_)) + { + createArchive(); + } + + void setCompression(const String & compression_method_, int compression_level_) override; + void setFormatAndSettings(Archive archive_) override; + void inferCompressionFromPath(); +}; +} +#endif diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index 03623c78624..782602091ac 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -1,6 +1,6 @@ -#include -#include #include +#include +#include #include @@ -8,8 +8,8 @@ namespace DB { namespace ErrorCodes { - extern const int CANNOT_UNPACK_ARCHIVE; - extern const int SUPPORT_IS_DISABLED; +extern const int CANNOT_UNPACK_ARCHIVE; +extern const int SUPPORT_IS_DISABLED; } @@ -25,16 +25,8 @@ std::shared_ptr createArchiveReader( [[maybe_unused]] size_t archive_size) { using namespace std::literals; - static constexpr std::array tar_extensions - { - ".tar"sv, - ".tar.gz"sv, - ".tgz"sv, - ".tar.zst"sv, - ".tzst"sv, - ".tar.xz"sv, - ".tar.bz2"sv - }; + static constexpr std::array tar_extensions{ + ".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) { diff --git a/src/IO/Archives/createArchiveWriter.cpp b/src/IO/Archives/createArchiveWriter.cpp index 2fb56e14bd3..fc13712ff59 100644 --- a/src/IO/Archives/createArchiveWriter.cpp +++ b/src/IO/Archives/createArchiveWriter.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -32,10 +33,13 @@ createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique #endif } //todo add support for extentsions i.e .gz - else if (path_to_archive.ends_with(".tar")) + else if (std::any_of( + TarArchiveWriter::TarExtensions.begin(), + TarArchiveWriter::TarExtensions.end(), + [&](const auto extension) { return path_to_archive.ends_with(extension); })) { #if USE_LIBARCHIVE - return std::make_shared(path_to_archive, std::move(archive_write_buffer)); + return std::make_shared(path_to_archive, std::move(archive_write_buffer)); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); #endif diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp index 3ac9bba5e5d..c8f8609047c 100644 --- a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp @@ -6,7 +6,8 @@ namespace DB bool hasRegisteredArchiveFileExtension(const String & path) { - return path.ends_with(".zip") || path.ends_with(".zipx") || path.ends_with(".tar"); + return path.ends_with(".zip") || path.ends_with(".zipx") || path.ends_with(".tar") || path.ends_with(".tar.gz") + || path.ends_with(".tar.bz2") || path.ends_with(".tar.lzma"); } } diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index dae1a2d5f9d..f794daa4b10 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -398,9 +398,9 @@ TEST_P(ArchiveReaderAndWriterTest, Password) { auto writer = createArchiveWriter(getPathToArchive()); //don't support passwords for tar archives - if(getPathToArchive().ends_with(".tar")) + if(getPathToArchive().ends_with(".tar") || getPathToArchive().ends_with(".tar.gz") || getPathToArchive().ends_with(".tar.bz2") || getPathToArchive().ends_with(".tar.lzma")) { - expectException(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for tar archives", + expectException(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for libarchive", [&]{ writer->setPassword("a.txt"); }); writer->finalize(); } @@ -675,6 +675,9 @@ namespace #endif #if USE_LIBARCHIVE ".tar", + ".tar.gz", + ".tar.bz2", + ".tar.lzma", #endif }; } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 70b328e525a..95ebefa3f4f 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -607,13 +607,61 @@ def test_tar_archive(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" +def test_tar_bz2_archive(): + backup_name = f"Disk('backups', 'archive.tar.bz2')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + assert os.path.isfile(get_path_to_backup(backup_name)) + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + +def test_tar_gz_archive(): + backup_name = f"Disk('backups', 'archive.tar.gz')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + assert os.path.isfile(get_path_to_backup(backup_name)) + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + +def test_tar_lzma_archive(): + backup_name = f"Disk('backups', 'archive.tar.lzma')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + assert os.path.isfile(get_path_to_backup(backup_name)) + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + def test_tar_archive_with_password(): backup_name = f"Disk('backups', 'archive_with_password.tar')" create_and_fill_table() assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" - expected_error = "Setting a password is not currently supported for tar archives" + expected_error = "Setting a password is not currently supported for libarchive" assert expected_error in instance.query_and_get_error( f"BACKUP TABLE test.table TO {backup_name} SETTINGS id='tar_archive_with_password', password='password123'" ) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 00ed5ee4d13..f17833e6c59 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -460,6 +460,28 @@ def test_backup_to_tar(): backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.tar', 'minio', 'minio123')" check_backup_and_restore(storage_policy, backup_destination) + +def test_backup_to_tar_gz(): + storage_policy = "default" + backup_name = new_backup_name() + backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.tar.gz', 'minio', 'minio123')" + check_backup_and_restore(storage_policy, backup_destination) + + +def test_backup_to_tar_bz2(): + storage_policy = "default" + backup_name = new_backup_name() + backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.tar.bz2', 'minio', 'minio123')" + check_backup_and_restore(storage_policy, backup_destination) + + +def test_backup_to_tar_lzma(): + storage_policy = "default" + backup_name = new_backup_name() + backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.tar.lzma', 'minio', 'minio123')" + check_backup_and_restore(storage_policy, backup_destination) + + def test_user_specific_auth(start_cluster): def create_user(user): node.query(f"CREATE USER {user}") From 79aec12771825c043343550c93daa8687aafffbd Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 20 Feb 2024 15:41:25 -0800 Subject: [PATCH 45/60] Update docs --- docs/en/operations/backup.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index e4933414647..31747fce613 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -170,7 +170,7 @@ RESTORE TABLE test.table PARTITIONS '2', '3' ### Backups as tar archives -Backups can also be stored as tar archives. The functionality is the same as for zip, except that compressing the archive or adding a password is not supported. +Backups can also be stored as tar archives. The functionality is the same as for zip, except that a password is not supported. Write a backup as a tar: ``` @@ -182,6 +182,14 @@ Corresponding restore: RESTORE TABLE test.table FROM Disk('backups', '1.tar') ``` +To change the compression method, the correct file suffix should be appended to the backup name. I.E to compress the tar archive using gzip: +``` +BACKUP TABLE test.table TO Disk('backups', '1.tar.gz') +``` + +The supported compression file suffixes are `.gz` `.bz2` and `.lzma`. + + ### Check the status of backups The backup command returns an `id` and `status`, and that `id` can be used to get the status of the backup. This is very useful to check the progress of long ASYNC backups. The example below shows a failure that happened when trying to overwrite an existing backup file: From 8bc968bb58154b15b50457ae8e4e7ee1b3e58248 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 20 Feb 2024 15:42:10 -0800 Subject: [PATCH 46/60] =?UTF-8?q?Fix=20tar=20integration=20test=C2=A0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- tests/integration/test_backup_restore_new/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 95ebefa3f4f..c16aeb1a188 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -679,7 +679,7 @@ def test_tar_archive_with_bad_compression_method(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" - expected_error = "Compressing tar archives is currently not supported" + expected_error = "Using compression_method and compression_level options are not supported for tar archives" assert expected_error in instance.query_and_get_error( f"BACKUP TABLE test.table TO {backup_name} SETTINGS id='tar_archive_with_bad_compression_method', compression_method='foobar'" ) From c02502b6f192233e96c31f8a89a3e798809eee6b Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 20 Feb 2024 15:53:55 -0800 Subject: [PATCH 47/60] Removed unused error code --- src/IO/Archives/TarArchiveWriter.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/Archives/TarArchiveWriter.cpp b/src/IO/Archives/TarArchiveWriter.cpp index 7bed0fd32a4..1c60a31e7af 100644 --- a/src/IO/Archives/TarArchiveWriter.cpp +++ b/src/IO/Archives/TarArchiveWriter.cpp @@ -5,7 +5,6 @@ namespace DB { namespace ErrorCodes { -extern const int CANNOT_PACK_ARCHIVE; extern const int NOT_IMPLEMENTED; } void TarArchiveWriter::setCompression(const String & compression_method_, int compression_level_) From f7ab6b9e37156cb1eb8c9776fb2f8fc373b890d9 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Tue, 20 Feb 2024 17:19:19 -0800 Subject: [PATCH 48/60] Move tar compression extension definitions --- src/IO/Archives/TarArchiveWriter.h | 2 -- src/IO/Archives/createArchiveWriter.cpp | 6 +++--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/IO/Archives/TarArchiveWriter.h b/src/IO/Archives/TarArchiveWriter.h index 553f68202f2..c2fe4c2a8a8 100644 --- a/src/IO/Archives/TarArchiveWriter.h +++ b/src/IO/Archives/TarArchiveWriter.h @@ -12,8 +12,6 @@ using namespace std::literals; class TarArchiveWriter : public LibArchiveWriter { public: - static constexpr std::array TarExtensions{".tar"sv, ".tar.gz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; - explicit TarArchiveWriter(const String & path_to_archive_, std::unique_ptr archive_write_buffer_) : LibArchiveWriter(path_to_archive_, std::move(archive_write_buffer_)) { diff --git a/src/IO/Archives/createArchiveWriter.cpp b/src/IO/Archives/createArchiveWriter.cpp index fc13712ff59..2a423784f4e 100644 --- a/src/IO/Archives/createArchiveWriter.cpp +++ b/src/IO/Archives/createArchiveWriter.cpp @@ -24,6 +24,8 @@ std::shared_ptr createArchiveWriter(const String & path_to_archi std::shared_ptr createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique_ptr archive_write_buffer) { + using namespace std::literals; + static constexpr std::array tar_extensions{".tar"sv, ".tar.gz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) { #if USE_MINIZIP @@ -34,9 +36,7 @@ createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique } //todo add support for extentsions i.e .gz else if (std::any_of( - TarArchiveWriter::TarExtensions.begin(), - TarArchiveWriter::TarExtensions.end(), - [&](const auto extension) { return path_to_archive.ends_with(extension); })) + tar_extensions.begin(), tar_extensions.end(), [&](const auto extension) { return path_to_archive.ends_with(extension); })) { #if USE_LIBARCHIVE return std::make_shared(path_to_archive, std::move(archive_write_buffer)); From 84a3a05eb40ce8286cf93cefb7b545090483c0bd Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 21 Feb 2024 08:26:17 -0500 Subject: [PATCH 49/60] Update src/IO/Archives/LibArchiveReader.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 7fa9b16b974..ef787c9ecb2 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -458,7 +458,7 @@ std::vector LibArchiveReader::getAllFiles(NameFilter filter) return handle.getAllFiles(filter); } -void LibArchiveReader::setPassword([[maybe_unused]] const String & password_) +void LibArchiveReader::setPassword(const String & password_) { if (password_.empty()) return; From f145b494c087ddbc8b476e15efbc9b39ccd352c1 Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 21 Feb 2024 08:26:42 -0500 Subject: [PATCH 50/60] Update src/IO/Archives/LibArchiveWriter.h Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveWriter.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveWriter.h b/src/IO/Archives/LibArchiveWriter.h index d42db73b3e8..4466f8fb6c5 100644 --- a/src/IO/Archives/LibArchiveWriter.h +++ b/src/IO/Archives/LibArchiveWriter.h @@ -53,7 +53,7 @@ protected: using Archive = struct archive *; using Entry = struct archive_entry *; - //derived classes must call createArcive. createArchive calls initArchive + /// derived classes must call createArcive. createArchive calls initArchive void createArchive(); virtual void setFormatAndSettings(Archive) = 0; From 6b32271b88ef9e67065237167f6403279b535b65 Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 21 Feb 2024 08:30:01 -0500 Subject: [PATCH 51/60] Update src/IO/Archives/LibArchiveWriter.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveWriter.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 84aed5315b7..a073dd249fa 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -25,8 +25,7 @@ void checkResultCodeImpl(int code, const String & filename) { if (code == ARCHIVE_OK) return; - String message = "LibArchive Code = " + std::to_string(code); - throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't pack archive: {}, filename={}", message, quoteString(filename)); + throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't pack archive: LibArchive Code = {}, filename={}", code, quoteString(filename)); } } From 5963b67dfff808e0bfca51a262a885bd54edf785 Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 21 Feb 2024 08:30:16 -0500 Subject: [PATCH 52/60] Update src/IO/Archives/LibArchiveWriter.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveWriter.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index a073dd249fa..cbe12d10ff1 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -35,7 +35,6 @@ class LibArchiveWriter::StreamInfo public: explicit StreamInfo(std::unique_ptr archive_write_buffer_) : archive_write_buffer(std::move(archive_write_buffer_)) { } - static ssize_t memory_write([[maybe_unused]] struct archive * archive, void * client_data, const void * buff, size_t length) { auto * stream_info = reinterpret_cast(client_data); From 0fe15f0a22b1f9f2668f3861b9d8b5cec9027c2a Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 21 Feb 2024 08:30:31 -0500 Subject: [PATCH 53/60] Update src/IO/Archives/LibArchiveWriter.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveWriter.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index cbe12d10ff1..a9e05bd2095 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -210,9 +210,8 @@ void LibArchiveWriter::endWritingFile() void LibArchiveWriter::startWritingFile() { std::lock_guard lock{mutex}; - if (is_writing_file) + if (std::exchange(is_writing_file, true)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot write two files to a tar archive in parallel"); - is_writing_file = true; } void LibArchiveWriter::finalize() From 1ce625647944397522609f6842d299bdaa912240 Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 21 Feb 2024 08:30:42 -0500 Subject: [PATCH 54/60] Update src/IO/Archives/LibArchiveWriter.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index a9e05bd2095..547b8a440ab 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -92,7 +92,7 @@ private: { throw Exception( ErrorCodes::CANNOT_PACK_ARCHIVE, - "Couldn't pack tar archive: Failed to write all bytes, {} of {} , filename={}", + "Couldn't pack tar archive: Failed to write all bytes, {} of {}, filename={}", written, to_write, quoteString(filename)); From f34ef716f082e9ff5fce1bd45eca26ba382e29cc Mon Sep 17 00:00:00 2001 From: josh-hildred Date: Wed, 21 Feb 2024 08:31:30 -0500 Subject: [PATCH 55/60] Update src/IO/Archives/LibArchiveWriter.cpp Co-authored-by: Antonio Andelic --- src/IO/Archives/LibArchiveWriter.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index 547b8a440ab..b55c61900d1 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -34,7 +34,6 @@ class LibArchiveWriter::StreamInfo { public: explicit StreamInfo(std::unique_ptr archive_write_buffer_) : archive_write_buffer(std::move(archive_write_buffer_)) { } - static ssize_t memory_write([[maybe_unused]] struct archive * archive, void * client_data, const void * buff, size_t length) { auto * stream_info = reinterpret_cast(client_data); From e6134bbb7b25dc6c14386bfd73030f9dcdea0359 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 21 Feb 2024 05:42:20 -0800 Subject: [PATCH 56/60] Address review comments --- src/IO/Archives/LibArchiveReader.cpp | 2 +- src/IO/Archives/LibArchiveWriter.cpp | 2 +- src/IO/Archives/createArchiveWriter.cpp | 2 -- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index ef787c9ecb2..b0910b1e209 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -23,7 +23,7 @@ class LibArchiveReader::StreamInfo public: explicit StreamInfo(std::unique_ptr read_buffer_) : read_buffer(std::move(read_buffer_)) { } - static ssize_t read([[maybe_unused]] struct archive * a, void * client_data, const void ** buff) + static ssize_t read(struct archive *, void * client_data, const void ** buff) { auto * read_stream = reinterpret_cast(client_data); *buff = reinterpret_cast(read_stream->buf); diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index b55c61900d1..ddde7652b13 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -34,7 +34,7 @@ class LibArchiveWriter::StreamInfo { public: explicit StreamInfo(std::unique_ptr archive_write_buffer_) : archive_write_buffer(std::move(archive_write_buffer_)) { } - static ssize_t memory_write([[maybe_unused]] struct archive * archive, void * client_data, const void * buff, size_t length) + static ssize_t memory_write(struct archive *, void * client_data, const void * buff, size_t length) { auto * stream_info = reinterpret_cast(client_data); stream_info->archive_write_buffer->write(reinterpret_cast(buff), length); diff --git a/src/IO/Archives/createArchiveWriter.cpp b/src/IO/Archives/createArchiveWriter.cpp index 2a423784f4e..e78f9fbdab6 100644 --- a/src/IO/Archives/createArchiveWriter.cpp +++ b/src/IO/Archives/createArchiveWriter.cpp @@ -34,7 +34,6 @@ createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif } - //todo add support for extentsions i.e .gz else if (std::any_of( tar_extensions.begin(), tar_extensions.end(), [&](const auto extension) { return path_to_archive.ends_with(extension); })) { @@ -47,5 +46,4 @@ createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique else throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Cannot determine the type of archive {}", path_to_archive); } - } From ca9a6f809f1aa2134b15d2a0844e911bcb7c2c6a Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Wed, 21 Feb 2024 15:35:30 -0800 Subject: [PATCH 57/60] add support for .tar.zst and tar.xz --- contrib/libarchive-cmake/CMakeLists.txt | 2 +- docs/en/operations/backup.md | 2 +- src/IO/Archives/LibArchiveWriter.cpp | 12 +- src/IO/Archives/LibArchiveWriter.h | 4 +- src/IO/Archives/TarArchiveWriter.cpp | 18 +- src/IO/Archives/TarArchiveWriter.h | 2 +- src/IO/Archives/createArchiveWriter.cpp | 3 +- .../hasRegisteredArchiveFileExtension.cpp | 3 +- .../tests/gtest_archive_reader_and_writer.cpp | 157 +++++++++--------- .../test_backup_restore_new/test.py | 32 ++++ .../test_backup_restore_s3/test.py | 14 ++ 11 files changed, 153 insertions(+), 96 deletions(-) diff --git a/contrib/libarchive-cmake/CMakeLists.txt b/contrib/libarchive-cmake/CMakeLists.txt index cd5658b7086..e89770da5f6 100644 --- a/contrib/libarchive-cmake/CMakeLists.txt +++ b/contrib/libarchive-cmake/CMakeLists.txt @@ -157,7 +157,7 @@ if (TARGET ch_contrib::zlib) endif() if (TARGET ch_contrib::zstd) - target_compile_definitions(_libarchive PUBLIC HAVE_ZSTD_H=1 HAVE_LIBZSTD=1) + target_compile_definitions(_libarchive PUBLIC HAVE_ZSTD_H=1 HAVE_LIBZSTD=1 HAVE_LIBZSTD_COMPRESSOR=1) target_link_libraries(_libarchive PRIVATE ch_contrib::zstd) endif() diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 31747fce613..2d9bf2a2ee8 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -187,7 +187,7 @@ To change the compression method, the correct file suffix should be appended to BACKUP TABLE test.table TO Disk('backups', '1.tar.gz') ``` -The supported compression file suffixes are `.gz` `.bz2` and `.lzma`. +The supported compression file suffixes are `tar.gz`, `.tgz` `tar.bz2`, `tar.lzma`, `.tar.zst`, `.tzst` and `.tar.xz`. ### Check the status of backups diff --git a/src/IO/Archives/LibArchiveWriter.cpp b/src/IO/Archives/LibArchiveWriter.cpp index ddde7652b13..55bc4c1f88c 100644 --- a/src/IO/Archives/LibArchiveWriter.cpp +++ b/src/IO/Archives/LibArchiveWriter.cpp @@ -25,7 +25,8 @@ void checkResultCodeImpl(int code, const String & filename) { if (code == ARCHIVE_OK) return; - throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't pack archive: LibArchive Code = {}, filename={}", code, quoteString(filename)); + throw Exception( + ErrorCodes::CANNOT_PACK_ARCHIVE, "Couldn't pack archive: LibArchive Code = {}, filename={}", code, quoteString(filename)); } } @@ -168,11 +169,14 @@ void LibArchiveWriter::createArchive() { std::lock_guard lock{mutex}; archive = archive_write_new(); - setFormatAndSettings(archive); - //this allows use to write directly to a writer buffer rather than an intermediate buffer in LibArchive - //archive_write_set_bytes_per_block(a, 0); + setFormatAndSettings(); if (stream_info) + { + //This allows use to write directly to a writebuffer rather than an intermediate buffer in libarchive. + //This has to be set otherwise zstd breaks due to extra bytes being written at the end of the archive. + archive_write_set_bytes_per_block(archive, 0); archive_write_open2(archive, stream_info.get(), nullptr, &StreamInfo::memory_write, nullptr, nullptr); + } else archive_write_open_filename(archive, path_to_archive.c_str()); } diff --git a/src/IO/Archives/LibArchiveWriter.h b/src/IO/Archives/LibArchiveWriter.h index 4466f8fb6c5..f54a8ce2367 100644 --- a/src/IO/Archives/LibArchiveWriter.h +++ b/src/IO/Archives/LibArchiveWriter.h @@ -53,9 +53,9 @@ protected: using Archive = struct archive *; using Entry = struct archive_entry *; - /// derived classes must call createArcive. createArchive calls initArchive + /// derived classes must call createArchive. CreateArchive calls setFormatAndSettings. void createArchive(); - virtual void setFormatAndSettings(Archive) = 0; + virtual void setFormatAndSettings() = 0; Archive archive = nullptr; String path_to_archive; diff --git a/src/IO/Archives/TarArchiveWriter.cpp b/src/IO/Archives/TarArchiveWriter.cpp index 1c60a31e7af..d390af89537 100644 --- a/src/IO/Archives/TarArchiveWriter.cpp +++ b/src/IO/Archives/TarArchiveWriter.cpp @@ -6,6 +6,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; +extern const int CANNOT_PACK_ARCHIVE; } void TarArchiveWriter::setCompression(const String & compression_method_, int compression_level_) { @@ -16,21 +17,26 @@ void TarArchiveWriter::setCompression(const String & compression_method_, int co ErrorCodes::NOT_IMPLEMENTED, "Using compression_method and compression_level options are not supported for tar archives"); } -void TarArchiveWriter::setFormatAndSettings(Archive archive_) +void TarArchiveWriter::setFormatAndSettings() { - archive_write_set_format_pax_restricted(archive_); + archive_write_set_format_pax_restricted(archive); inferCompressionFromPath(); } void TarArchiveWriter::inferCompressionFromPath() { - if (path_to_archive.ends_with(".gz")) + if (path_to_archive.ends_with(".tar.gz") || path_to_archive.ends_with(".tgz")) archive_write_add_filter_gzip(archive); - else if (path_to_archive.ends_with(".bz2")) + else if (path_to_archive.ends_with(".tar.bz2")) archive_write_add_filter_bzip2(archive); - else if (path_to_archive.ends_with(".lzma")) + else if (path_to_archive.ends_with(".tar.lzma")) archive_write_add_filter_lzma(archive); - //else path ends in .tar and we dont do any compression + else if (path_to_archive.ends_with(".tar.zst") || path_to_archive.ends_with(".tzst")) + archive_write_add_filter_zstd(archive); + else if (path_to_archive.ends_with(".tar.xz")) + archive_write_add_filter_xz(archive); + else if (!path_to_archive.ends_with(".tar")) + throw Exception(ErrorCodes::CANNOT_PACK_ARCHIVE, "Unknown compression format"); } } #endif diff --git a/src/IO/Archives/TarArchiveWriter.h b/src/IO/Archives/TarArchiveWriter.h index c2fe4c2a8a8..b04ec4083af 100644 --- a/src/IO/Archives/TarArchiveWriter.h +++ b/src/IO/Archives/TarArchiveWriter.h @@ -19,7 +19,7 @@ public: } void setCompression(const String & compression_method_, int compression_level_) override; - void setFormatAndSettings(Archive archive_) override; + void setFormatAndSettings() override; void inferCompressionFromPath(); }; } diff --git a/src/IO/Archives/createArchiveWriter.cpp b/src/IO/Archives/createArchiveWriter.cpp index e78f9fbdab6..9a169587088 100644 --- a/src/IO/Archives/createArchiveWriter.cpp +++ b/src/IO/Archives/createArchiveWriter.cpp @@ -25,7 +25,8 @@ std::shared_ptr createArchiveWriter(const String & path_to_archive, [[maybe_unused]] std::unique_ptr archive_write_buffer) { using namespace std::literals; - static constexpr std::array tar_extensions{".tar"sv, ".tar.gz"sv, ".tar.bz2"sv, ".tar.lzma"sv}; + static constexpr std::array tar_extensions{ + ".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.bz2"sv, ".tar.lzma"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv}; if (path_to_archive.ends_with(".zip") || path_to_archive.ends_with(".zipx")) { #if USE_MINIZIP diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp index c8f8609047c..b24b3436dc0 100644 --- a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp @@ -7,7 +7,8 @@ namespace DB bool hasRegisteredArchiveFileExtension(const String & path) { return path.ends_with(".zip") || path.ends_with(".zipx") || path.ends_with(".tar") || path.ends_with(".tar.gz") - || path.ends_with(".tar.bz2") || path.ends_with(".tar.lzma"); + || path.ends_with(".tar.bz2") || path.ends_with(".tar.lzma") || path.ends_with(".tar.zst") || path.ends_with(".tzst") + || path.ends_with(".tgz") || path.ends_with(".tar.xz"); } } diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index f794daa4b10..898c7017e7d 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -1,30 +1,29 @@ #include #include "config.h" +#include +#include #include #include #include #include #include -#include #include +#include #include #include -#include #include +#include #include #include -#include #include -#include -#include - +#include namespace DB::ErrorCodes { - extern const int CANNOT_UNPACK_ARCHIVE; - extern const int LOGICAL_ERROR; +extern const int CANNOT_UNPACK_ARCHIVE; +extern const int LOGICAL_ERROR; } namespace fs = std::filesystem; @@ -53,7 +52,8 @@ bool createArchiveWithFiles(const std::string & archivename, const std::map @@ -118,11 +117,13 @@ TEST_P(ArchiveReaderAndWriterTest, EmptyArchive) EXPECT_FALSE(reader->fileExists("nofile.txt")); - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "File 'nofile.txt' was not found in archive", - [&]{ reader->getFileInfo("nofile.txt"); }); + expectException( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, "File 'nofile.txt' was not found in archive", [&] { reader->getFileInfo("nofile.txt"); }); - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "File 'nofile.txt' was not found in archive", - [&]{ reader->readFile("nofile.txt", /*throw_on_not_found=*/true); }); + expectException( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, + "File 'nofile.txt' was not found in archive", + [&] { reader->readFile("nofile.txt", /*throw_on_not_found=*/true); }); EXPECT_EQ(reader->firstFile(), nullptr); } @@ -186,11 +187,9 @@ TEST_P(ArchiveReaderAndWriterTest, SingleFileInArchive) auto enumerator = reader->firstFile(); ASSERT_NE(enumerator, nullptr); EXPECT_FALSE(enumerator->nextFile()); - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "No current file", - [&]{ enumerator->getFileName(); }); + expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "No current file", [&] { enumerator->getFileName(); }); - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "No current file", - [&] { reader->readFile(std::move(enumerator)); }); + expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "No current file", [&] { reader->readFile(std::move(enumerator)); }); } } @@ -280,7 +279,7 @@ TEST_P(ArchiveReaderAndWriterTest, TwoFilesInArchive) enumerator = reader->nextFile(std::move(in)); EXPECT_EQ(enumerator, nullptr); } - + // Get all files one last time files = reader->getAllFiles(); EXPECT_EQ(files.size(), 2); @@ -313,7 +312,8 @@ TEST_P(ArchiveReaderAndWriterTest, InMemory) ASSERT_FALSE(fs::exists(getPathToArchive())); /// Read the archive. - auto read_archive_func = [&]() -> std::unique_ptr { return std::make_unique(archive_in_memory); }; + auto read_archive_func + = [&]() -> std::unique_ptr { return std::make_unique(archive_in_memory); }; auto reader = createArchiveReader(getPathToArchive(), read_archive_func, archive_in_memory.size()); ASSERT_TRUE(reader->fileExists("a.txt")); @@ -355,15 +355,13 @@ TEST_P(ArchiveReaderAndWriterTest, ManyFilesInMemory) { auto writer = createArchiveWriter(getPathToArchive(), std::make_unique(archive_in_memory)); { - for(int i = 0; i < files; i++) + for (int i = 0; i < files; i++) { auto filename = std::format("{}.txt", i); auto contents = std::format("The contents of {}.txt", i); auto out = writer->writeFile(filename, times * contents.size()); - for(int j = 0; j < times; j++) - { + for (int j = 0; j < times; j++) writeString(contents, *out); - } out->finalize(); } } @@ -374,10 +372,11 @@ TEST_P(ArchiveReaderAndWriterTest, ManyFilesInMemory) ASSERT_FALSE(fs::exists(getPathToArchive())); /// Read the archive. - auto read_archive_func = [&]() -> std::unique_ptr { return std::make_unique(archive_in_memory); }; + auto read_archive_func + = [&]() -> std::unique_ptr { return std::make_unique(archive_in_memory); }; auto reader = createArchiveReader(getPathToArchive(), read_archive_func, archive_in_memory.size()); - for(int i = 0; i < files; i++) + for (int i = 0; i < files; i++) { auto filename = std::format("{}.txt", i); auto contents = std::format("The contents of {}.txt", i); @@ -386,28 +385,29 @@ TEST_P(ArchiveReaderAndWriterTest, ManyFilesInMemory) { auto in = reader->readFile(filename, /*throw_on_not_found=*/true); - for(int j = 0; j < times; j++) - { + for (int j = 0; j < times; j++) ASSERT_TRUE(checkString(String(contents), *in)); - } } } } TEST_P(ArchiveReaderAndWriterTest, Password) -{ +{ auto writer = createArchiveWriter(getPathToArchive()); //don't support passwords for tar archives - if(getPathToArchive().ends_with(".tar") || getPathToArchive().ends_with(".tar.gz") || getPathToArchive().ends_with(".tar.bz2") || getPathToArchive().ends_with(".tar.lzma")) + if (getPathToArchive().ends_with(".tar") || getPathToArchive().ends_with(".tar.gz") || getPathToArchive().ends_with(".tar.bz2") + || getPathToArchive().ends_with(".tar.lzma") || getPathToArchive().ends_with(".tar.zst") || getPathToArchive().ends_with(".tar.xz")) { - expectException(ErrorCodes::NOT_IMPLEMENTED, "Setting a password is not currently supported for libarchive", - [&]{ writer->setPassword("a.txt"); }); + expectException( + ErrorCodes::NOT_IMPLEMENTED, + "Setting a password is not currently supported for libarchive", + [&] { writer->setPassword("a.txt"); }); writer->finalize(); } else { - /// Make an archive. - std::string_view contents = "The contents of a.txt"; + /// Make an archive. + std::string_view contents = "The contents of a.txt"; { writer->setPassword("Qwe123"); { @@ -422,14 +422,14 @@ TEST_P(ArchiveReaderAndWriterTest, Password) auto reader = createArchiveReader(getPathToArchive()); /// Try to read without a password. - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Password is required", - [&]{ reader->readFile("a.txt", /*throw_on_not_found=*/true); }); + expectException( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Password is required", [&] { reader->readFile("a.txt", /*throw_on_not_found=*/true); }); { /// Try to read with a wrong password. reader->setPassword("123Qwe"); - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Wrong password", - [&]{ reader->readFile("a.txt", /*throw_on_not_found=*/true); }); + expectException( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Wrong password", [&] { reader->readFile("a.txt", /*throw_on_not_found=*/true); }); } { @@ -446,8 +446,7 @@ TEST_P(ArchiveReaderAndWriterTest, Password) TEST_P(ArchiveReaderAndWriterTest, ArchiveNotExist) { - expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open", - [&]{ createArchiveReader(getPathToArchive()); }); + expectException(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open", [&] { createArchiveReader(getPathToArchive()); }); } @@ -459,15 +458,13 @@ TEST_P(ArchiveReaderAndWriterTest, ManyFilesOnDisk) { auto writer = createArchiveWriter(getPathToArchive()); { - for(int i = 0; i < files; i++) + for (int i = 0; i < files; i++) { auto filename = std::format("{}.txt", i); auto contents = std::format("The contents of {}.txt", i); auto out = writer->writeFile(filename, times * contents.size()); - for(int j = 0; j < times; j++) - { + for (int j = 0; j < times; j++) writeString(contents, *out); - } out->finalize(); } } @@ -480,7 +477,7 @@ TEST_P(ArchiveReaderAndWriterTest, ManyFilesOnDisk) /// Read the archive. auto reader = createArchiveReader(getPathToArchive()); - for(int i = 0; i < files; i++) + for (int i = 0; i < files; i++) { auto filename = std::format("{}.txt", i); auto contents = std::format("The contents of {}.txt", i); @@ -489,10 +486,8 @@ TEST_P(ArchiveReaderAndWriterTest, ManyFilesOnDisk) { auto in = reader->readFile(filename, /*throw_on_not_found=*/true); - for(int j = 0; j < times; j++) - { + for (int j = 0; j < times; j++) ASSERT_TRUE(checkString(String(contents), *in)); - } } } } @@ -506,10 +501,8 @@ TEST_P(ArchiveReaderAndWriterTest, LargeFile) auto writer = createArchiveWriter(getPathToArchive()); { auto out = writer->writeFile("a.txt", times * contents.size()); - for(int i = 0; i < times; i++) - { + for (int i = 0; i < times; i++) writeString(contents, *out); - } out->finalize(); } writer->finalize(); @@ -526,10 +519,8 @@ TEST_P(ArchiveReaderAndWriterTest, LargeFile) { auto in = reader->readFile("a.txt", /*throw_on_not_found=*/true); - for(int i = 0; i < times; i++) - { + for (int i = 0; i < times; i++) ASSERT_TRUE(checkString(String(contents), *in)); - } } { @@ -543,7 +534,8 @@ TEST_P(ArchiveReaderAndWriterTest, LargeFile) } } -TEST(TarArchiveReaderTest, FileExists) { +TEST(TarArchiveReaderTest, FileExists) +{ String archive_path = "archive.tar"; String filename = "file.txt"; String contents = "test"; @@ -554,7 +546,8 @@ TEST(TarArchiveReaderTest, FileExists) { fs::remove(archive_path); } -TEST(TarArchiveReaderTest, ReadFile) { +TEST(TarArchiveReaderTest, ReadFile) +{ String archive_path = "archive.tar"; String filename = "file.txt"; String contents = "test"; @@ -568,7 +561,8 @@ TEST(TarArchiveReaderTest, ReadFile) { fs::remove(archive_path); } -TEST(TarArchiveReaderTest, ReadTwoFiles) { +TEST(TarArchiveReaderTest, ReadTwoFiles) +{ String archive_path = "archive.tar"; String file1 = "file1.txt"; String contents1 = "test1"; @@ -584,14 +578,15 @@ TEST(TarArchiveReaderTest, ReadTwoFiles) { readStringUntilEOF(str, *in); EXPECT_EQ(str, contents1); in = reader->readFile(file2, /*throw_on_not_found=*/true); - + readStringUntilEOF(str, *in); EXPECT_EQ(str, contents2); fs::remove(archive_path); } -TEST(TarArchiveReaderTest, CheckFileInfo) { +TEST(TarArchiveReaderTest, CheckFileInfo) +{ String archive_path = "archive.tar"; String filename = "file.txt"; String contents = "test"; @@ -604,7 +599,8 @@ TEST(TarArchiveReaderTest, CheckFileInfo) { fs::remove(archive_path); } -TEST(SevenZipArchiveReaderTest, FileExists) { +TEST(SevenZipArchiveReaderTest, FileExists) +{ String archive_path = "archive.7z"; String filename = "file.txt"; String contents = "test"; @@ -615,7 +611,8 @@ TEST(SevenZipArchiveReaderTest, FileExists) { fs::remove(archive_path); } -TEST(SevenZipArchiveReaderTest, ReadFile) { +TEST(SevenZipArchiveReaderTest, ReadFile) +{ String archive_path = "archive.7z"; String filename = "file.txt"; String contents = "test"; @@ -629,7 +626,8 @@ TEST(SevenZipArchiveReaderTest, ReadFile) { fs::remove(archive_path); } -TEST(SevenZipArchiveReaderTest, CheckFileInfo) { +TEST(SevenZipArchiveReaderTest, CheckFileInfo) +{ String archive_path = "archive.7z"; String filename = "file.txt"; String contents = "test"; @@ -642,7 +640,8 @@ TEST(SevenZipArchiveReaderTest, CheckFileInfo) { fs::remove(archive_path); } -TEST(SevenZipArchiveReaderTest, ReadTwoFiles) { +TEST(SevenZipArchiveReaderTest, ReadTwoFiles) +{ String archive_path = "archive.7z"; String file1 = "file1.txt"; String contents1 = "test1"; @@ -658,28 +657,28 @@ TEST(SevenZipArchiveReaderTest, ReadTwoFiles) { readStringUntilEOF(str, *in); EXPECT_EQ(str, contents1); in = reader->readFile(file2, /*throw_on_not_found=*/true); - + readStringUntilEOF(str, *in); EXPECT_EQ(str, contents2); fs::remove(archive_path); } - namespace { - const char * supported_archive_file_exts[] = - { - #if USE_MINIZIP - ".zip", - #endif - #if USE_LIBARCHIVE - ".tar", - ".tar.gz", - ".tar.bz2", - ".tar.lzma", - #endif - }; +const char * supported_archive_file_exts[] = { +#if USE_MINIZIP + ".zip", +#endif +#if USE_LIBARCHIVE + ".tar", + ".tar.gz", + ".tar.bz2", + ".tar.lzma", + ".tar.zst", + ".tar.xz", +#endif +}; } INSTANTIATE_TEST_SUITE_P(All, ArchiveReaderAndWriterTest, ::testing::ValuesIn(supported_archive_file_exts)); diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index c16aeb1a188..34ffdf7a8df 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -655,6 +655,38 @@ def test_tar_lzma_archive(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" +def test_tar_zst_archive(): + backup_name = f"Disk('backups', 'archive.tar.zst')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + assert os.path.isfile(get_path_to_backup(backup_name)) + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + +def test_tar_xz_archive(): + backup_name = f"Disk('backups', 'archive.tar.xz')" + create_and_fill_table() + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + assert os.path.isfile(get_path_to_backup(backup_name)) + + instance.query("DROP TABLE test.table") + assert instance.query("EXISTS test.table") == "0\n" + + instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + + def test_tar_archive_with_password(): backup_name = f"Disk('backups', 'archive_with_password.tar')" create_and_fill_table() diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index f17833e6c59..4d3ee8200a3 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -482,6 +482,20 @@ def test_backup_to_tar_lzma(): check_backup_and_restore(storage_policy, backup_destination) +def test_backup_to_tar_zst(): + storage_policy = "default" + backup_name = new_backup_name() + backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.tar.zst', 'minio', 'minio123')" + check_backup_and_restore(storage_policy, backup_destination) + + +def test_backup_to_tar_xz(): + storage_policy = "default" + backup_name = new_backup_name() + backup_destination = f"S3('http://minio1:9001/root/data/backups/{backup_name}.tar.xz', 'minio', 'minio123')" + check_backup_and_restore(storage_policy, backup_destination) + + def test_user_specific_auth(start_cluster): def create_user(user): node.query(f"CREATE USER {user}") From fe672935331be559a621e87160e433da031e47e4 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 26 Feb 2024 13:07:32 -0800 Subject: [PATCH 58/60] Add support for lzma --- src/IO/Archives/LibArchiveReader.cpp | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index b0910b1e209..73f7234a5c3 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -197,13 +197,18 @@ private: archive_read_support_filter_xz(archive); archive_read_support_filter_lz4(archive); archive_read_support_filter_zstd(archive); + archive_read_support_filter_lzma(archive); // Support tar, 7zip and zip archive_read_support_format_tar(archive); archive_read_support_format_7zip(archive); archive_read_support_format_zip(archive); - if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open archive {}: {}", quoteString(path_to_archive), archive_error_string(archive)); + if (archive_read_open(archive, read_stream_, nullptr, StreamInfo::read, nullptr) != ARCHIVE_OK) + throw Exception( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, + "Couldn't open archive {}: {}", + quoteString(path_to_archive), + archive_error_string(archive)); } catch (...) { @@ -219,8 +224,17 @@ private: auto * archive = archive_read_new(); try { - archive_read_support_filter_all(archive); - archive_read_support_format_all(archive); + // Support for bzip2, gzip, lzip, xz, zstd and lz4 + archive_read_support_filter_bzip2(archive); + archive_read_support_filter_gzip(archive); + archive_read_support_filter_xz(archive); + archive_read_support_filter_lz4(archive); + archive_read_support_filter_zstd(archive); + archive_read_support_filter_lzma(archive); + // Support tar, 7zip and zip + archive_read_support_format_tar(archive); + archive_read_support_format_7zip(archive); + archive_read_support_format_zip(archive); if (archive_read_open_filename(archive, path_to_archive_.c_str(), 10240) != ARCHIVE_OK) throw Exception( ErrorCodes::CANNOT_UNPACK_ARCHIVE, From 58a865852862ca972073b0fe0562cf1b98148328 Mon Sep 17 00:00:00 2001 From: Joshua Hildred Date: Mon, 26 Feb 2024 13:12:35 -0800 Subject: [PATCH 59/60] Use std::array and std::any_of --- src/IO/Archives/LibArchiveReader.cpp | 8 +++----- src/IO/Archives/hasRegisteredArchiveFileExtension.cpp | 9 +++++---- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 73f7234a5c3..bec7f587180 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -317,10 +317,7 @@ public: } bool checkIfActuallySeekable() override { return false; } - off_t getPosition() override - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); - } + off_t getPosition() override { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); } String getFileName() const override { return handle.getFileName(); } size_t getFileSize() override { return handle.getFileInfo().uncompressed_size; } @@ -456,7 +453,8 @@ std::unique_ptr LibArchiveReader::currentFile( { if (!dynamic_cast(read_buffer.get())) throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong ReadBuffer passed to nextFile()"); - auto read_buffer_from_libarchive = std::unique_ptr(static_cast(read_buffer.release())); + auto read_buffer_from_libarchive + = std::unique_ptr(static_cast(read_buffer.release())); auto handle = std::move(*read_buffer_from_libarchive).releaseHandle(); return std::make_unique(std::move(handle)); } diff --git a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp index b24b3436dc0..2a979f500f7 100644 --- a/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp +++ b/src/IO/Archives/hasRegisteredArchiveFileExtension.cpp @@ -6,9 +6,10 @@ namespace DB bool hasRegisteredArchiveFileExtension(const String & path) { - return path.ends_with(".zip") || path.ends_with(".zipx") || path.ends_with(".tar") || path.ends_with(".tar.gz") - || path.ends_with(".tar.bz2") || path.ends_with(".tar.lzma") || path.ends_with(".tar.zst") || path.ends_with(".tzst") - || path.ends_with(".tgz") || path.ends_with(".tar.xz"); + using namespace std::literals; + static constexpr std::array archive_extensions{ + ".zip"sv, ".zipx"sv, ".tar"sv, ".tar.gz"sv, ".tgz"sv, ".tar.bz2"sv, ".tar.lzma"sv, ".tar.zst"sv, ".tzst"sv, ".tar.xz"sv}; + return std::any_of( + archive_extensions.begin(), archive_extensions.end(), [&](const auto extension) { return path.ends_with(extension); }); } - } From bd423c5ac5a5fc40cd1758181db54d725597f615 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 29 Feb 2024 15:39:27 +0000 Subject: [PATCH 60/60] CI: fix for build job cache record status #job_package_release #job_style_check --- tests/ci/ci.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index b7f76a2e652..ae9360a6d67 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -953,10 +953,18 @@ def _mark_success_action( # FIXME: find generic design for propagating and handling job status (e.g. stop using statuses in GH api) # now job ca be build job w/o status data, any other job that exit with 0 with or w/o status data if CI_CONFIG.is_build_job(job): - # there is no status for build jobs - # create dummy success to mark it as done + # there is no CommitStatus for build jobs + # create dummy status relying on JobReport # FIXME: consider creating commit status for build jobs too, to treat everything the same way - CommitStatusData(SUCCESS, "dummy description", "dummy_url").dump_status() + job_report = JobReport.load() if JobReport.exist() else None + if job_report and job_report.status == SUCCESS: + CommitStatusData( + SUCCESS, + "dummy description", + "dummy_url", + pr_num=pr_info.number, + sha=pr_info.sha, + ).dump_status() job_status = None if CommitStatusData.exist():