diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 64dc9049bc2..f9765c1d57b 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -46,42 +46,35 @@ At a minimum, the following information should be added (but add more as needed) **NOTE:** If your merge the PR with modified CI you **MUST KNOW** what you are doing **NOTE:** Checked options will be applied if set before CI RunConfig/PrepareRunConfig step - -#### Run these jobs only (required builds will be added automatically): -- [ ] Integration Tests -- [ ] Stateless tests -- [ ] Stateful tests -- [ ] Unit tests -- [ ] Performance tests -- [ ] All with aarch64 -- [ ] All with ASAN -- [ ] All with TSAN -- [ ] All with Analyzer -- [ ] All with Azure -- [ ] Add your option here - -#### Deny these jobs: -- [ ] Fast test -- [ ] Integration Tests -- [ ] Stateless tests -- [ ] Stateful tests -- [ ] Performance tests -- [ ] All with ASAN -- [ ] All with TSAN -- [ ] All with MSAN -- [ ] All with UBSAN -- [ ] All with Coverage -- [ ] All with Aarch64 - -#### Extra options: +- [ ] Allow: Integration Tests +- [ ] Allow: Stateless tests +- [ ] Allow: Stateful tests +- [ ] Allow: Unit tests +- [ ] Allow: Performance tests +- [ ] Allow: All with aarch64 +- [ ] Allow: All with ASAN +- [ ] Allow: All with TSAN +- [ ] Allow: All with Analyzer +- [ ] Allow: All with Azure +- [ ] Allow: Add your option here +--- +- [ ] Exclude: Fast test +- [ ] Exclude: Integration Tests +- [ ] Exclude: Stateless tests +- [ ] Exclude: Stateful tests +- [ ] Exclude: Performance tests +- [ ] Exclude: All with ASAN +- [ ] Exclude: All with TSAN +- [ ] Exclude: All with MSAN +- [ ] Exclude: All with UBSAN +- [ ] Exclude: All with Coverage +- [ ] Exclude: All with Aarch64 +--- - [ ] do not test (only style check) - [ ] disable merge-commit (no merge from master before tests) - [ ] disable CI cache (job reuse) - -#### Only specified batches in multi-batch jobs: -- [ ] 1 -- [ ] 2 -- [ ] 3 -- [ ] 4 - +- [ ] allow: batch 1 for multi-batch jobs +- [ ] allow: batch 2 +- [ ] allow: batch 3 +- [ ] allow: batch 4, 5 and 6 diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 11ec484d208..c2a893a8e99 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -27,15 +27,16 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 sync_pr.py --merge || : - - name: Python unit tests - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - echo "Testing the main ci directory" - python3 -m unittest discover -s . -p 'test_*.py' - for dir in *_lambda/; do - echo "Testing $dir" - python3 -m unittest discover -s "$dir" -p 'test_*.py' - done +# Runs in MQ: +# - name: Python unit tests +# run: | +# cd "$GITHUB_WORKSPACE/tests/ci" +# echo "Testing the main ci directory" +# python3 -m unittest discover -s . -p 'test_*.py' +# for dir in *_lambda/; do +# echo "Testing $dir" +# python3 -m unittest discover -s "$dir" -p 'test_*.py' +# done - name: PrepareRunConfig id: runconfig run: | @@ -135,7 +136,7 @@ jobs: MarkReleaseReady: if: ${{ !failure() && !cancelled() }} - needs: [RunConfig, Builds_1] + needs: [RunConfig, Builds_1, Builds_2] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Debug @@ -162,7 +163,7 @@ jobs: python3 mark_release_ready.py FinishCheck: - if: ${{ !failure() && !cancelled() }} + if: ${{ !cancelled() }} needs: [RunConfig, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index aa570c3ce2f..7d22554473e 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -33,9 +33,12 @@ jobs: clear-repository: true # to ensure correct digests fetch-depth: 0 # to get a version filter: tree:0 - - name: Cancel Sync PR workflow + - name: Cancel previous Sync PR workflow run: | python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --cancel-previous-run + - name: Set pending Sync status + run: | + python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --set-pending-status - name: Labels check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -177,7 +180,7 @@ jobs: ################################# Stage Final ################################# # FinishCheck: - if: ${{ !failure() && !cancelled() }} + if: ${{ !cancelled() }} needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: diff --git a/.github/workflows/reusable_build.yml b/.github/workflows/reusable_build.yml index 80d78d93e1b..5e254d785ec 100644 --- a/.github/workflows/reusable_build.yml +++ b/.github/workflows/reusable_build.yml @@ -33,6 +33,10 @@ name: Build ClickHouse additional_envs: description: additional ENV variables to setup the job type: string + secrets: + secret_envs: + description: if given, it's passed to the environments + required: false jobs: Build: @@ -54,6 +58,7 @@ jobs: run: | cat >> "$GITHUB_ENV" << 'EOF' ${{inputs.additional_envs}} + ${{secrets.secret_envs}} DOCKER_TAG<args[1].safeGet(); break; default: - UNREACHABLE(); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected operation: {}", operation); } auto response = client->zookeeper->reconfig(joining, leaving, new_members); diff --git a/programs/main.cpp b/programs/main.cpp index bc8476e4ce4..c270388f17f 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -155,8 +155,8 @@ auto instructionFailToString(InstructionFail fail) ret("AVX2"); case InstructionFail::AVX512: ret("AVX512"); +#undef ret } - UNREACHABLE(); } diff --git a/src/Access/AccessEntityIO.cpp b/src/Access/AccessEntityIO.cpp index b0dfd74c53b..1b073329296 100644 --- a/src/Access/AccessEntityIO.cpp +++ b/src/Access/AccessEntityIO.cpp @@ -144,8 +144,7 @@ AccessEntityPtr deserializeAccessEntity(const String & definition, const String catch (Exception & e) { e.addMessage("Could not parse " + file_path); - e.rethrow(); - UNREACHABLE(); + throw; } } diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index c10931f554c..2127f4ada70 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -258,7 +258,7 @@ namespace case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel(); case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel(); } - UNREACHABLE(); + chassert(false); } } diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 8e51481e415..8d4e7d3073e 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -257,8 +257,7 @@ std::vector IAccessStorage::insert(const std::vector & mu } e.addMessage("After successfully inserting {}/{}: {}", successfully_inserted.size(), multiple_entities.size(), successfully_inserted_str); } - e.rethrow(); - UNREACHABLE(); + throw; } } @@ -361,8 +360,7 @@ std::vector IAccessStorage::remove(const std::vector & ids, bool thr } e.addMessage("After successfully removing {}/{}: {}", removed_names.size(), ids.size(), removed_names_str); } - e.rethrow(); - UNREACHABLE(); + throw; } } @@ -458,8 +456,7 @@ std::vector IAccessStorage::update(const std::vector & ids, const Up } e.addMessage("After successfully updating {}/{}: {}", names_of_updated.size(), ids.size(), names_of_updated_str); } - e.rethrow(); - UNREACHABLE(); + throw; } } diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index d4fb7afcb78..930b2c6ce73 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -60,14 +60,13 @@ struct GroupArrayTrait template constexpr const char * getNameByTrait() { - if (Trait::last) + if constexpr (Trait::last) return "groupArrayLast"; - if (Trait::sampler == Sampler::NONE) - return "groupArray"; - else if (Trait::sampler == Sampler::RNG) - return "groupArraySample"; - - UNREACHABLE(); + switch (Trait::sampler) + { + case Sampler::NONE: return "groupArray"; + case Sampler::RNG: return "groupArraySample"; + } } template diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index bed10333af0..a9dd53a75e8 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -414,7 +414,6 @@ public: break; return (i == events_size) ? base - i : unmatched_idx; } - UNREACHABLE(); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h index 58aaddf357a..2ce03c530c2 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.h +++ b/src/AggregateFunctions/AggregateFunctionSum.h @@ -463,7 +463,6 @@ public: return "sumWithOverflow"; else if constexpr (Type == AggregateFunctionTypeSumKahan) return "sumKahan"; - UNREACHABLE(); } explicit AggregateFunctionSum(const DataTypes & argument_types_) diff --git a/src/Analyzer/ValidationUtils.cpp b/src/Analyzer/ValidationUtils.cpp index b7f9307e4b3..c142a0c7cc0 100644 --- a/src/Analyzer/ValidationUtils.cpp +++ b/src/Analyzer/ValidationUtils.cpp @@ -331,6 +331,9 @@ void validateAggregates(const QueryTreeNodePtr & query_node, AggregatesValidatio if (query_node_typed.hasOrderBy()) validate_group_by_columns_visitor.visit(query_node_typed.getOrderByNode()); + if (query_node_typed.hasInterpolate()) + validate_group_by_columns_visitor.visit(query_node_typed.getInterpolate()); + validate_group_by_columns_visitor.visit(query_node_typed.getProjectionNode()); } diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 331cace67d7..8c3c5327e94 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -30,7 +29,7 @@ namespace ErrorCodes } BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( - StorageAzureBlob::Configuration configuration_, + const StorageAzureConfiguration & configuration_, bool allow_azure_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, @@ -39,15 +38,14 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.getConnectionURL().toString(), false, false} , configuration(configuration_) { - auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + auto client_ptr = configuration.createClient(/* is_readonly */false, /* attempt_to_create_container */true); client_ptr->SetClickhouseOptions(Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}); - object_storage = std::make_unique( - "BackupReaderAzureBlobStorage", - std::move(client_ptr), - StorageAzureBlob::createSettings(context_), - configuration.container, - configuration.getConnectionURL().toString()); + object_storage = std::make_unique("BackupReaderAzureBlobStorage", + std::move(client_ptr), + configuration.createSettings(context_), + configuration_.container, + configuration.getConnectionURL().toString()); client = object_storage->getAzureBlobStorageClient(); auto settings_copy = *object_storage->getSettings(); @@ -121,7 +119,7 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( - StorageAzureBlob::Configuration configuration_, + const StorageAzureConfiguration & configuration_, bool allow_azure_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, @@ -131,13 +129,13 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.getConnectionURL().toString(), false, false} , configuration(configuration_) { - auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false, attempt_to_create_container); + auto client_ptr = configuration.createClient(/* is_readonly */false, attempt_to_create_container); client_ptr->SetClickhouseOptions(Azure::Storage::Blobs::ClickhouseClientOptions{.IsClientForDisk=true}); object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), - StorageAzureBlob::createSettings(context_), - configuration_.container, + configuration.createSettings(context_), + configuration.container, configuration_.getConnectionURL().toString()); client = object_storage->getAzureBlobStorageClient(); auto settings_copy = *object_storage->getSettings(); @@ -145,8 +143,13 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( settings = std::make_unique(settings_copy); } -void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, - bool copy_encrypted, UInt64 start_pos, UInt64 length) +void BackupWriterAzureBlobStorage::copyFileFromDisk( + const String & path_in_backup, + DiskPtr src_disk, + const String & src_path, + bool copy_encrypted, + UInt64 start_pos, + UInt64 length) { /// Use the native copy as a more optimal way to copy a file from AzureBlobStorage to AzureBlobStorage if it's possible. auto source_data_source_description = src_disk->getDataSourceDescription(); @@ -196,10 +199,16 @@ void BackupWriterAzureBlobStorage::copyFile(const String & destination, const St threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure")); } -void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) +void BackupWriterAzureBlobStorage::copyDataToFile( + const String & path_in_backup, + const CreateReadBufferFunction & create_read_buffer, + UInt64 start_pos, + UInt64 length) { - copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, fs::path(configuration.blob_path) / path_in_backup, settings, - threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure")); + copyDataToAzureBlobStorageFile( + create_read_buffer, start_pos, length, client, configuration.container, + fs::path(configuration.blob_path) / path_in_backup, settings, + threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure")); } BackupWriterAzureBlobStorage::~BackupWriterAzureBlobStorage() = default; @@ -217,7 +226,7 @@ UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) object_storage->listObjects(key,children,/*max_keys*/0); if (children.empty()) throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Object must exist"); - return children[0].metadata.size_bytes; + return children[0]->metadata->size_bytes; } std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String & file_name, size_t /*expected_file_size*/) diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 3a909ab684a..61688107839 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -5,8 +5,8 @@ #if USE_AZURE_BLOB_STORAGE #include #include -#include #include +#include namespace DB @@ -17,24 +17,30 @@ class BackupReaderAzureBlobStorage : public BackupReaderDefault { public: BackupReaderAzureBlobStorage( - StorageAzureBlob::Configuration configuration_, + const StorageAzureConfiguration & configuration_, bool allow_azure_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + ~BackupReaderAzureBlobStorage() override; bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr readFile(const String & file_name) override; - void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; + void copyFileToDisk( + const String & path_in_backup, + size_t file_size, + bool encrypted_in_backup, + DiskPtr destination_disk, + const String & destination_path, + WriteMode write_mode) override; private: const DataSourceDescription data_source_description; std::shared_ptr client; - StorageAzureBlob::Configuration configuration; + StorageAzureConfiguration configuration; std::unique_ptr object_storage; std::shared_ptr settings; }; @@ -43,21 +49,32 @@ class BackupWriterAzureBlobStorage : public BackupWriterDefault { public: BackupWriterAzureBlobStorage( - StorageAzureBlob::Configuration configuration_, + const StorageAzureConfiguration & configuration_, bool allow_azure_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, bool attempt_to_create_container); + ~BackupWriterAzureBlobStorage() override; bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr writeFile(const String & file_name) override; - void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override; - void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, - bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void copyDataToFile( + const String & path_in_backup, + const CreateReadBufferFunction & create_read_buffer, + UInt64 start_pos, + UInt64 length) override; + + void copyFileFromDisk( + const String & path_in_backup, + DiskPtr src_disk, + const String & src_path, + bool copy_encrypted, + UInt64 start_pos, + UInt64 length) override; void copyFile(const String & destination, const String & source, size_t size) override; @@ -67,9 +84,10 @@ public: private: std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; void removeFilesBatch(const Strings & file_names); + const DataSourceDescription data_source_description; std::shared_ptr client; - StorageAzureBlob::Configuration configuration; + StorageAzureConfiguration configuration; std::unique_ptr object_storage; std::shared_ptr settings; }; diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 15860363615..92e208ba464 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -131,10 +131,10 @@ BackupReaderS3::BackupReaderS3( : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderS3")) , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} - , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup)) + , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup).value_or(S3Settings{})) { auto & request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context_->getSettingsRef()); + request_settings.updateFromSettingsIfChanged(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint request_settings.allow_native_copy = allow_s3_native_copy; client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); @@ -222,10 +222,10 @@ BackupWriterS3::BackupWriterS3( : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3")) , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} - , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup)) + , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString(), context_->getUserName(), /*ignore_user=*/is_internal_backup).value_or(S3Settings{})) { auto & request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context_->getSettingsRef()); + request_settings.updateFromSettingsIfChanged(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint request_settings.allow_native_copy = allow_s3_native_copy; request_settings.setStorageClassName(storage_class_name); diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 8b05965f472..81e3c104da1 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -5,11 +5,11 @@ #if USE_AZURE_BLOB_STORAGE #include -#include #include #include #include #include +#include #include #endif @@ -49,7 +49,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) const String & id_arg = params.backup_info.id_arg; const auto & args = params.backup_info.args; - StorageAzureBlob::Configuration configuration; + StorageAzureConfiguration configuration; if (!id_arg.empty()) { @@ -81,10 +81,11 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) } if (args.size() > 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Backup AzureBlobStorage requires 1 or 2 arguments: named_collection, [filename]"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Backup AzureBlobStorage requires 1 or 2 arguments: named_collection, [filename]"); if (args.size() == 1) - configuration.blob_path = args[0].safeGet(); + configuration.setPath(args[0].safeGet()); } else @@ -116,12 +117,16 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) } BackupImpl::ArchiveParams archive_params; - if (hasRegisteredArchiveFileExtension(configuration.blob_path)) + if (hasRegisteredArchiveFileExtension(configuration.getPath())) { if (params.is_internal_backup) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Using archives with backups on clusters is disabled"); - archive_params.archive_name = removeFileNameFromURL(configuration.blob_path); + auto path = configuration.getPath(); + auto filename = removeFileNameFromURL(path); + configuration.setPath(path); + + archive_params.archive_name = filename; archive_params.compression_method = params.compression_method; archive_params.compression_level = params.compression_level; archive_params.password = params.password; diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4e8946facda..f2e10a27b75 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -115,8 +115,11 @@ if (TARGET ch_contrib::nats_io) add_headers_and_sources(dbms Storages/NATS) endif() -add_headers_and_sources(dbms Storages/DataLakes) -add_headers_and_sources(dbms Storages/DataLakes/Iceberg) +add_headers_and_sources(dbms Storages/ObjectStorage) +add_headers_and_sources(dbms Storages/ObjectStorage/Azure) +add_headers_and_sources(dbms Storages/ObjectStorage/S3) +add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) +add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) if (TARGET ch_contrib::amqp_cpp) @@ -144,7 +147,6 @@ if (TARGET ch_contrib::azure_sdk) endif() if (TARGET ch_contrib::hdfs) - add_headers_and_sources(dbms Storages/HDFS) add_headers_and_sources(dbms Disks/ObjectStorages/HDFS) endif() diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index b557edc3e12..e73ac307a35 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -168,6 +168,9 @@ M(ObjectStorageS3Threads, "Number of threads in the S3ObjectStorage thread pool.") \ M(ObjectStorageS3ThreadsActive, "Number of threads in the S3ObjectStorage thread pool running a task.") \ M(ObjectStorageS3ThreadsScheduled, "Number of queued or active jobs in the S3ObjectStorage thread pool.") \ + M(StorageObjectStorageThreads, "Number of threads in the remote table engines thread pools.") \ + M(StorageObjectStorageThreadsActive, "Number of threads in the remote table engines thread pool running a task.") \ + M(StorageObjectStorageThreadsScheduled, "Number of queued or active jobs in remote table engines thread pool.") \ M(ObjectStorageAzureThreads, "Number of threads in the AzureObjectStorage thread pool.") \ M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \ M(ObjectStorageAzureThreadsScheduled, "Number of queued or active jobs in the AzureObjectStorage thread pool.") \ diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 392ee64dcbf..c87d44a4b95 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -41,7 +41,6 @@ UInt8 getDayOfWeek(const cctz::civil_day & date) case cctz::weekday::saturday: return 6; case cctz::weekday::sunday: return 7; } - UNREACHABLE(); } inline cctz::time_point lookupTz(const cctz::time_zone & cctz_time_zone, const cctz::civil_day & date) diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index 22c7db504c3..1548d5cf9a5 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -34,8 +34,6 @@ Int64 IntervalKind::toAvgNanoseconds() const default: return toAvgSeconds() * NANOSECONDS_PER_SECOND; } - - UNREACHABLE(); } Int32 IntervalKind::toAvgSeconds() const @@ -54,7 +52,6 @@ Int32 IntervalKind::toAvgSeconds() const case IntervalKind::Kind::Quarter: return 7889238; /// Exactly 1/4 of a year. case IntervalKind::Kind::Year: return 31556952; /// The average length of a Gregorian year is equal to 365.2425 days } - UNREACHABLE(); } Float64 IntervalKind::toSeconds() const @@ -80,7 +77,6 @@ Float64 IntervalKind::toSeconds() const default: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not possible to get precise number of seconds in non-precise interval"); } - UNREACHABLE(); } bool IntervalKind::isFixedLength() const @@ -99,7 +95,6 @@ bool IntervalKind::isFixedLength() const case IntervalKind::Kind::Quarter: case IntervalKind::Kind::Year: return false; } - UNREACHABLE(); } IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds) @@ -141,7 +136,6 @@ const char * IntervalKind::toKeyword() const case IntervalKind::Kind::Quarter: return "QUARTER"; case IntervalKind::Kind::Year: return "YEAR"; } - UNREACHABLE(); } @@ -161,7 +155,6 @@ const char * IntervalKind::toLowercasedKeyword() const case IntervalKind::Kind::Quarter: return "quarter"; case IntervalKind::Kind::Year: return "year"; } - UNREACHABLE(); } @@ -192,7 +185,6 @@ const char * IntervalKind::toDateDiffUnit() const case IntervalKind::Kind::Year: return "year"; } - UNREACHABLE(); } @@ -223,7 +215,6 @@ const char * IntervalKind::toNameOfFunctionToIntervalDataType() const case IntervalKind::Kind::Year: return "toIntervalYear"; } - UNREACHABLE(); } @@ -257,7 +248,6 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const case IntervalKind::Kind::Year: return "toYear"; } - UNREACHABLE(); } diff --git a/src/Common/TargetSpecific.cpp b/src/Common/TargetSpecific.cpp index 49f396c0926..8540c9a9986 100644 --- a/src/Common/TargetSpecific.cpp +++ b/src/Common/TargetSpecific.cpp @@ -54,8 +54,6 @@ String toString(TargetArch arch) case TargetArch::AMXTILE: return "amxtile"; case TargetArch::AMXINT8: return "amxint8"; } - - UNREACHABLE(); } } diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 6a63d484cd9..23b41f23bde 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -75,7 +75,6 @@ const char * TasksStatsCounters::metricsProviderString(MetricsProvider provider) case MetricsProvider::Netlink: return "netlink"; } - UNREACHABLE(); } bool TasksStatsCounters::checkIfAvailable() diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 7d2602bde1e..7cca262baca 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -146,8 +146,6 @@ const char * errorMessage(Error code) case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; case Error::ZNOTREADONLY: return "State-changing request is passed to read-only server"; } - - UNREACHABLE(); } bool isHardwareError(Error zk_return_code) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 2185d32e47a..ed7498b1ac9 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1259,11 +1259,13 @@ void ZooKeeper::initFeatureFlags() void ZooKeeper::executeGenericRequest( const ZooKeeperRequestPtr & request, - ResponseCallback callback) + ResponseCallback callback, + WatchCallbackPtr watch) { RequestInfo request_info; request_info.request = request; request_info.callback = callback; + request_info.watch = watch; pushRequest(std::move(request_info)); } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index cf331a03d06..8fdf0f97d9d 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -139,7 +139,8 @@ public: void executeGenericRequest( const ZooKeeperRequestPtr & request, - ResponseCallback callback); + ResponseCallback callback, + WatchCallbackPtr watch = nullptr); /// See the documentation about semantics of these methods in IKeeper class. diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 7e0653c69f8..f1b5b24e866 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -466,7 +466,6 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); return; } - UNREACHABLE(); } void CompressionCodecDeflateQpl::flushAsynchronousDecompressRequests() diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index e6e8db4c699..cbd8cd57a62 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -21,6 +21,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + /** NOTE DoubleDelta is surprisingly bad name. The only excuse is that it comes from an academic paper. * Most people will think that "double delta" is just applying delta transform twice. * But in fact it is something more than applying delta transform twice. @@ -142,9 +147,9 @@ namespace ErrorCodes { extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; - extern const int BAD_ARGUMENTS; extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; extern const int ILLEGAL_CODEC_PARAMETER; + extern const int LOGICAL_ERROR; } namespace @@ -163,9 +168,8 @@ inline Int64 getMaxValueForByteSize(Int8 byte_size) case sizeof(UInt64): return std::numeric_limits::max(); default: - assert(false && "only 1, 2, 4 and 8 data sizes are supported"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "only 1, 2, 4 and 8 data sizes are supported"); } - UNREACHABLE(); } struct WriteSpec diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp index e3642913a7a..05211af6704 100644 --- a/src/Coordination/KeeperReconfiguration.cpp +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -5,6 +5,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining) { ClusterUpdateActions out; @@ -79,7 +85,7 @@ String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateA new_config.emplace_back(RaftServerConfig{*cfg->get_server(priority->id)}); } else - UNREACHABLE(); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected update"); } for (const auto & item : cfg->get_servers()) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 8d21ce2ab01..736a01443ce 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -990,7 +990,7 @@ KeeperServer::ConfigUpdateState KeeperServer::applyConfigUpdate( raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); return Accepted; } - UNREACHABLE(); + std::unreachable(); } ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index bae6328a328..4b14b038852 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -145,9 +146,10 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes + std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage + mutable std::mutex keeper_dispatcher_mutex; mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); - }; ContextData::ContextData() = default; @@ -453,6 +455,19 @@ std::shared_ptr Context::getZooKeeper() const throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot connect to ZooKeeper from Keeper"); } +const StorageS3Settings & Context::getStorageS3Settings() const +{ + std::lock_guard lock(shared->mutex); + + if (!shared->storage_s3_settings) + { + const auto & config = shared->config ? *shared->config : Poco::Util::Application::instance().config(); + shared->storage_s3_settings.emplace().loadFromConfig("s3", config, getSettingsRef()); + } + + return *shared->storage_s3_settings; +} + const ServerSettings & Context::getServerSettings() const { return shared->server_settings; diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index 3df3649c498..7e4d1794f7d 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -37,6 +37,7 @@ class FilesystemCacheLog; class FilesystemReadPrefetchesLog; class BlobStorageLog; class IOUringReader; +class StorageS3Settings; /// A small class which owns ContextShared. /// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete. @@ -162,6 +163,10 @@ public: zkutil::ZooKeeperPtr getZooKeeper() const; + const StorageS3Settings & getStorageS3Settings() const; + + const String & getUserName() const { static std::string user; return user; } + const ServerSettings & getServerSettings() const; bool hasTraceCollector() const; diff --git a/src/Core/Field.h b/src/Core/Field.h index 4424d669c4d..710614cd0a0 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -667,8 +667,6 @@ public: case Types::AggregateFunctionState: return f(field.template get()); case Types::CustomType: return f(field.template get()); } - - UNREACHABLE(); } String dump() const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 97b6abc3913..d9165e1bcce 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -116,6 +116,12 @@ class IColumn; M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, azure_allow_parallel_part_upload, true, "Use multiple threads for azure multipart upload.", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, azure_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, s3_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in S3 table engine", 0) \ + M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ + M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ + M(Bool, s3_validate_request_settings, true, "Validate S3 request settings", 0) \ M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 30000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ @@ -128,6 +134,7 @@ class IColumn; M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ M(Bool, hdfs_skip_empty_files, false, "Allow to skip empty files in hdfs table engine", 0) \ + M(Bool, azure_skip_empty_files, false, "Allow to skip empty files in azure table engine", 0) \ M(UInt64, hsts_max_age, 0, "Expired time for hsts. 0 means disable HSTS.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 53871f5546e..3faa9b12602 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,12 +85,20 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated functions"}, + {"24.6", {{"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, + {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, + {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, + {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, + {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, + {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, + {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + }}, + {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "A new setting."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, + {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, + {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, {"http_max_chunk_size", 0, 0, "Internal limitation"}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index dbe27a5f3f6..bbb1d1a6cd1 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -36,7 +36,6 @@ String ISerialization::kindToString(Kind kind) case Kind::SPARSE: return "Sparse"; } - UNREACHABLE(); } ISerialization::Kind ISerialization::stringToKind(const String & str) diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 1de7f80f512..060991d1290 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 3433698a162..cb34f7932c3 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -140,7 +140,6 @@ private: case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE: return "REMOTE_FS_READ_AND_PUT_IN_CACHE"; } - UNREACHABLE(); } size_t first_offset = 0; diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 96ba6acefff..3da6d843991 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include namespace Poco { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index bee8e206ec4..e7ecf7cd515 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -79,14 +79,14 @@ private: for (const auto & blob : blobs_list) { - batch.emplace_back( + batch.emplace_back(std::make_shared( blob.Name, ObjectMetadata{ static_cast(blob.BlobSize), Poco::Timestamp::fromEpochTime( std::chrono::duration_cast( static_cast(blob.Details.LastModified).time_since_epoch()).count()), - {}}); + {}})); } if (!blob_list_response.NextPageToken.HasValue() || blob_list_response.NextPageToken.Value().empty()) @@ -148,15 +148,15 @@ bool AzureObjectStorage::exists(const StoredObject & object) const return false; } -ObjectStorageIteratorPtr AzureObjectStorage::iterate(const std::string & path_prefix) const +ObjectStorageIteratorPtr AzureObjectStorage::iterate(const std::string & path_prefix, size_t max_keys) const { auto settings_ptr = settings.get(); auto client_ptr = client.get(); - return std::make_shared(path_prefix, client_ptr, settings_ptr->list_object_keys_size); + return std::make_shared(path_prefix, client_ptr, max_keys); } -void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { auto client_ptr = client.get(); @@ -179,19 +179,19 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith for (const auto & blob : blobs_list) { - children.emplace_back( + children.emplace_back(std::make_shared( blob.Name, ObjectMetadata{ static_cast(blob.BlobSize), Poco::Timestamp::fromEpochTime( std::chrono::duration_cast( static_cast(blob.Details.LastModified).time_since_epoch()).count()), - {}}); + {}})); } if (max_keys) { - int keys_left = max_keys - static_cast(children.size()); + size_t keys_left = max_keys - children.size(); if (keys_left <= 0) break; options.PageSizeHint = keys_left; @@ -346,10 +346,11 @@ void AzureObjectStorage::removeObjectsIfExist(const StoredObjects & objects) { auto client_ptr = client.get(); for (const auto & object : objects) + { removeObjectImpl(object, client_ptr, true); + } } - ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) const { auto client_ptr = client.get(); @@ -366,9 +367,9 @@ ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) c { result.attributes.emplace(); for (const auto & [key, value] : properties.Metadata) - (*result.attributes)[key] = value; + result.attributes[key] = value; } - result.last_modified.emplace(static_cast(properties.LastModified).time_since_epoch().count()); + result.last_modified = static_cast(properties.LastModified).time_since_epoch().count(); return result; } @@ -397,7 +398,9 @@ void AzureObjectStorage::copyObject( /// NOLINT dest_blob_client.CopyFromUri(source_blob_client.GetUrl(), copy_options); } -void AzureObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) +void AzureObjectStorage::applyNewSettings( + const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + ContextPtr context, const ApplyNewSettingsOptions &) { auto new_settings = getAzureBlobStorageSettings(config, config_prefix, context); settings.set(std::move(new_settings)); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index c3062def763..8ead696cf78 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -85,9 +85,9 @@ public: const String & object_namespace_, const String & description_); - void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; - ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const override; + ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const override; std::string getName() const override { return "AzureObjectStorage"; } @@ -144,7 +144,8 @@ public: void applyNewSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context) override; + ContextPtr context, + const ApplyNewSettingsOptions & options) override; String getObjectsNamespace() const override { return object_namespace ; } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index e3ab772e3b5..f2f33684fde 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -176,7 +176,7 @@ std::unique_ptr CachedObjectStorage::cloneObjectStorage( return object_storage->cloneObjectStorage(new_namespace, config, config_prefix, context); } -void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { object_storage->listObjects(path, children, max_keys); } @@ -192,9 +192,10 @@ void CachedObjectStorage::shutdown() } void CachedObjectStorage::applyNewSettings( - const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) + const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + ContextPtr context, const ApplyNewSettingsOptions & options) { - object_storage->applyNewSettings(config, config_prefix, context); + object_storage->applyNewSettings(config, config_prefix, context, options); } String CachedObjectStorage::getObjectsNamespace() const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 961c2709efc..a4d263e92eb 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -80,7 +80,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; ObjectMetadata getObjectMetadata(const std::string & path) const override; @@ -91,7 +91,8 @@ public: void applyNewSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context) override; + ContextPtr context, + const ApplyNewSettingsOptions & options) override; String getObjectsNamespace() const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index d4ff9bc0b79..abf0c1fad0b 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -544,7 +544,7 @@ void DiskObjectStorage::applyNewSettings( { /// FIXME we cannot use config_prefix that was passed through arguments because the disk may be wrapped with cache and we need another name const auto config_prefix = "storage_configuration.disks." + name; - object_storage->applyNewSettings(config, config_prefix, context_); + object_storage->applyNewSettings(config, config_prefix, context_, IObjectStorage::ApplyNewSettingsOptions{ .allow_client_change = true }); { std::unique_lock lock(resource_mutex); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 18a0377efe7..701c08b9a14 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -364,18 +364,18 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * for (const auto & object : objects) { - LOG_INFO(disk->log, "Calling restore for key for disk {}", object.relative_path); + LOG_INFO(disk->log, "Calling restore for key for disk {}", object->relative_path); /// Skip file operations objects. They will be processed separately. - if (object.relative_path.find("/operations/") != String::npos) + if (object->relative_path.find("/operations/") != String::npos) continue; - const auto [revision, _] = extractRevisionAndOperationFromKey(object.relative_path); + const auto [revision, _] = extractRevisionAndOperationFromKey(object->relative_path); /// Filter early if it's possible to get revision from key. if (revision > restore_information.revision) continue; - keys_names.push_back(object.relative_path); + keys_names.push_back(object->relative_path); } if (!keys_names.empty()) @@ -405,26 +405,20 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::processRestoreFiles( { for (const auto & key : keys) { - auto meta = source_object_storage->getObjectMetadata(key); - auto object_attributes = meta.attributes; + auto metadata = source_object_storage->getObjectMetadata(key); + auto object_attributes = metadata.attributes; String path; - if (object_attributes.has_value()) + /// Restore file if object has 'path' in metadata. + auto path_entry = object_attributes.find("path"); + if (path_entry == object_attributes.end()) { - /// Restore file if object has 'path' in metadata. - auto path_entry = object_attributes->find("path"); - if (path_entry == object_attributes->end()) - { - /// Such keys can remain after migration, we can skip them. - LOG_WARNING(disk->log, "Skip key {} because it doesn't have 'path' in metadata", key); - continue; - } - - path = path_entry->second; - } - else + /// Such keys can remain after migration, we can skip them. + LOG_WARNING(disk->log, "Skip key {} because it doesn't have 'path' in metadata", key); continue; + } + path = path_entry->second; disk->createDirectories(directoryPath(path)); auto object_key = ObjectStorageKey::createAsRelative(disk->object_key_prefix, shrinkKey(source_path, key)); @@ -436,7 +430,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::processRestoreFiles( source_object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, *disk->object_storage); auto tx = disk->metadata_storage->createTransaction(); - tx->addBlobToMetadata(path, object_key, meta.size_bytes); + tx->addBlobToMetadata(path, object_key, metadata.size_bytes); tx->commit(); LOG_TRACE(disk->log, "Restored file {}", path); @@ -475,10 +469,10 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject for (const auto & object : objects) { - const auto [revision, operation] = extractRevisionAndOperationFromKey(object.relative_path); + const auto [revision, operation] = extractRevisionAndOperationFromKey(object->relative_path); if (revision == UNKNOWN_REVISION) { - LOG_WARNING(disk->log, "Skip key {} with unknown revision", object.relative_path); + LOG_WARNING(disk->log, "Skip key {} with unknown revision", object->relative_path); continue; } @@ -491,7 +485,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject if (send_metadata) revision_counter = revision - 1; - auto object_attributes = *(source_object_storage->getObjectMetadata(object.relative_path).attributes); + auto object_attributes = source_object_storage->getObjectMetadata(object->relative_path).attributes; if (operation == rename) { auto from_path = object_attributes["from_path"]; diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index e717c88ed22..dcb2af9d4d3 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -1,12 +1,13 @@ #include #include -#include -#include +#include +#include -#include +#include #include #include +#include #if USE_HDFS @@ -18,28 +19,57 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int HDFS_ERROR; + extern const int ACCESS_DENIED; + extern const int LOGICAL_ERROR; } -void HDFSObjectStorage::shutdown() +void HDFSObjectStorage::initializeHDFSFS() const { + if (initialized) + return; + + std::lock_guard lock(init_mutex); + if (initialized) + return; + + hdfs_builder = createHDFSBuilder(url, config); + hdfs_fs = createHDFSFS(hdfs_builder.get()); + initialized = true; } -void HDFSObjectStorage::startup() +std::string HDFSObjectStorage::extractObjectKeyFromURL(const StoredObject & object) const { + /// This is very unfortunate, but for disk HDFS we made a mistake + /// and now its behaviour is inconsistent with S3 and Azure disks. + /// The mistake is that for HDFS we write into metadata files whole URL + data directory + key, + /// while for S3 and Azure we write there only data_directory + key. + /// This leads us into ambiguity that for StorageHDFS we have just key in object.remote_path, + /// but for DiskHDFS we have there URL as well. + auto path = object.remote_path; + if (path.starts_with(url)) + path = path.substr(url.size()); + if (path.starts_with("/")) + path.substr(1); + return path; } ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const { + initializeHDFSFS(); /// what ever data_source_description.description value is, consider that key as relative key - return ObjectStorageKey::createAsRelative(hdfs_root_path, getRandomASCIIString(32)); + chassert(data_directory.starts_with("/")); + return ObjectStorageKey::createAsRelative( + fs::path(url_without_path) / data_directory.substr(1), getRandomASCIIString(32)); } bool HDFSObjectStorage::exists(const StoredObject & object) const { - const auto & path = object.remote_path; - const size_t begin_of_path = path.find('/', path.find("//") + 2); - const String remote_fs_object_path = path.substr(begin_of_path); - return (0 == hdfsExists(hdfs_fs.get(), remote_fs_object_path.c_str())); + initializeHDFSFS(); + std::string path = object.remote_path; + if (path.starts_with(url_without_path)) + path = path.substr(url_without_path.size()); + + return (0 == hdfsExists(hdfs_fs.get(), path.c_str())); } std::unique_ptr HDFSObjectStorage::readObject( /// NOLINT @@ -48,7 +78,10 @@ std::unique_ptr HDFSObjectStorage::readObject( /// NOLIN std::optional, std::optional) const { - return std::make_unique(object.remote_path, object.remote_path, config, patchSettings(read_settings)); + initializeHDFSFS(); + auto path = extractObjectKeyFromURL(object); + return std::make_unique( + fs::path(url_without_path) / "", fs::path(data_directory) / path, config, patchSettings(read_settings)); } std::unique_ptr HDFSObjectStorage::readObjects( /// NOLINT @@ -57,18 +90,15 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI std::optional, std::optional) const { + initializeHDFSFS(); auto disk_read_settings = patchSettings(read_settings); auto read_buffer_creator = [this, disk_read_settings] (bool /* restricted_seek */, const StoredObject & object_) -> std::unique_ptr { - const auto & path = object_.remote_path; - size_t begin_of_path = path.find('/', path.find("//") + 2); - auto hdfs_path = path.substr(begin_of_path); - auto hdfs_uri = path.substr(0, begin_of_path); - + auto path = extractObjectKeyFromURL(object_); return std::make_unique( - hdfs_uri, hdfs_path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); + fs::path(url_without_path) / "", fs::path(data_directory) / path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); }; return std::make_unique( @@ -82,14 +112,21 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL size_t buf_size, const WriteSettings & write_settings) { + initializeHDFSFS(); if (attributes.has_value()) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects"); + std::string path = object.remote_path; + if (path.starts_with("/")) + path = path.substr(1); + if (!path.starts_with(url)) + path = fs::path(url) / path; + /// Single O_WRONLY in libhdfs adds O_TRUNC return std::make_unique( - object.remote_path, config, settings->replication, patchSettings(write_settings), buf_size, + path, config, settings->replication, patchSettings(write_settings), buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); } @@ -97,11 +134,13 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL /// Remove file. Throws exception if file doesn't exists or it's a directory. void HDFSObjectStorage::removeObject(const StoredObject & object) { - const auto & path = object.remote_path; - const size_t begin_of_path = path.find('/', path.find("//") + 2); + initializeHDFSFS(); + auto path = object.remote_path; + if (path.starts_with(url_without_path)) + path = path.substr(url_without_path.size()); /// Add path from root to file name - int res = hdfsDelete(hdfs_fs.get(), path.substr(begin_of_path).c_str(), 0); + int res = hdfsDelete(hdfs_fs.get(), path.c_str(), 0); if (res == -1) throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: {}", path); @@ -109,27 +148,85 @@ void HDFSObjectStorage::removeObject(const StoredObject & object) void HDFSObjectStorage::removeObjects(const StoredObjects & objects) { + initializeHDFSFS(); for (const auto & object : objects) removeObject(object); } void HDFSObjectStorage::removeObjectIfExists(const StoredObject & object) { + initializeHDFSFS(); if (exists(object)) removeObject(object); } void HDFSObjectStorage::removeObjectsIfExist(const StoredObjects & objects) { + initializeHDFSFS(); for (const auto & object : objects) removeObjectIfExists(object); } -ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string &) const +ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) const { - throw Exception( - ErrorCodes::UNSUPPORTED_METHOD, - "HDFS API doesn't support custom attributes/metadata for stored objects"); + initializeHDFSFS(); + auto * file_info = hdfsGetPathInfo(hdfs_fs.get(), path.data()); + if (!file_info) + throw Exception(ErrorCodes::HDFS_ERROR, + "Cannot get file info for: {}. Error: {}", path, hdfsGetLastError()); + + ObjectMetadata metadata; + metadata.size_bytes = static_cast(file_info->mSize); + metadata.last_modified = Poco::Timestamp::fromEpochTime(file_info->mLastMod); + + hdfsFreeFileInfo(file_info, 1); + return metadata; +} + +void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const +{ + initializeHDFSFS(); + LOG_TEST(log, "Trying to list files for {}", path); + + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(hdfs_fs.get(), path.data(), &ls.length); + + if (ls.file_info == nullptr && errno != ENOENT) // NOLINT + { + // ignore file not found exception, keep throw other exception, + // libhdfs3 doesn't have function to get exception type, so use errno. + throw Exception(ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", + path, String(hdfsGetLastError())); + } + + if (!ls.file_info && ls.length > 0) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); + } + + LOG_TEST(log, "Listed {} files for {}", ls.length, path); + + for (int i = 0; i < ls.length; ++i) + { + const String file_path = fs::path(ls.file_info[i].mName).lexically_normal(); + const bool is_directory = ls.file_info[i].mKind == 'D'; + if (is_directory) + { + listObjects(fs::path(file_path) / "", children, max_keys); + } + else + { + children.emplace_back(std::make_shared( + String(file_path), + ObjectMetadata{ + static_cast(ls.file_info[i].mSize), + Poco::Timestamp::fromEpochTime(ls.file_info[i].mLastMod), + {}})); + } + + if (max_keys && children.size() >= max_keys) + break; + } } void HDFSObjectStorage::copyObject( /// NOLINT @@ -139,6 +236,7 @@ void HDFSObjectStorage::copyObject( /// NOLINT const WriteSettings & write_settings, std::optional object_to_attributes) { + initializeHDFSFS(); if (object_to_attributes.has_value()) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, @@ -151,7 +249,10 @@ void HDFSObjectStorage::copyObject( /// NOLINT } -std::unique_ptr HDFSObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr) +std::unique_ptr HDFSObjectStorage::cloneObjectStorage( + const std::string &, + const Poco::Util::AbstractConfiguration &, + const std::string &, ContextPtr) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS object storage doesn't support cloning"); } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 66095eb9f8f..8aae90d0721 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include @@ -16,21 +16,13 @@ namespace DB struct HDFSObjectStorageSettings { - - HDFSObjectStorageSettings() = default; - - size_t min_bytes_for_seek; - int objects_chunk_size_to_delete; - int replication; - - HDFSObjectStorageSettings( - int min_bytes_for_seek_, - int objects_chunk_size_to_delete_, - int replication_) + HDFSObjectStorageSettings(int min_bytes_for_seek_, int replication_) : min_bytes_for_seek(min_bytes_for_seek_) - , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) , replication(replication_) {} + + size_t min_bytes_for_seek; + int replication; }; @@ -43,20 +35,29 @@ public: HDFSObjectStorage( const String & hdfs_root_path_, SettingsPtr settings_, - const Poco::Util::AbstractConfiguration & config_) + const Poco::Util::AbstractConfiguration & config_, + bool lazy_initialize) : config(config_) - , hdfs_builder(createHDFSBuilder(hdfs_root_path_, config)) - , hdfs_fs(createHDFSFS(hdfs_builder.get())) , settings(std::move(settings_)) - , hdfs_root_path(hdfs_root_path_) + , log(getLogger("HDFSObjectStorage(" + hdfs_root_path_ + ")")) { + const size_t begin_of_path = hdfs_root_path_.find('/', hdfs_root_path_.find("//") + 2); + url = hdfs_root_path_; + url_without_path = url.substr(0, begin_of_path); + if (begin_of_path < url.size()) + data_directory = url.substr(begin_of_path); + else + data_directory = "/"; + + if (!lazy_initialize) + initializeHDFSFS(); } std::string getName() const override { return "HDFSObjectStorage"; } - std::string getCommonKeyPrefix() const override { return hdfs_root_path; } + std::string getCommonKeyPrefix() const override { return url; } - std::string getDescription() const override { return hdfs_root_path; } + std::string getDescription() const override { return url; } ObjectStorageType getType() const override { return ObjectStorageType::HDFS; } @@ -100,9 +101,7 @@ public: const WriteSettings & write_settings, std::optional object_to_attributes = {}) override; - void shutdown() override; - - void startup() override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; String getObjectsNamespace() const override { return ""; } @@ -116,13 +115,28 @@ public: bool isRemote() const override { return true; } + void startup() override { } + + void shutdown() override { } + private: + void initializeHDFSFS() const; + std::string extractObjectKeyFromURL(const StoredObject & object) const; + const Poco::Util::AbstractConfiguration & config; - HDFSBuilderWrapper hdfs_builder; - HDFSFSPtr hdfs_fs; + mutable HDFSBuilderWrapper hdfs_builder; + mutable HDFSFSPtr hdfs_fs; + + mutable std::mutex init_mutex; + mutable std::atomic_bool initialized{false}; + SettingsPtr settings; - const std::string hdfs_root_path; + std::string url; + std::string url_without_path; + std::string data_directory; + + LoggerPtr log; }; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index accef9a08ab..fd1269df79b 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -25,16 +25,16 @@ bool IObjectStorage::existsOrHasAnyChild(const std::string & path) const return !files.empty(); } -void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata &, int) const +void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata &, size_t) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listObjects() is not supported"); } -ObjectStorageIteratorPtr IObjectStorage::iterate(const std::string & path_prefix) const +ObjectStorageIteratorPtr IObjectStorage::iterate(const std::string & path_prefix, size_t max_keys) const { RelativePathsWithMetadata files; - listObjects(path_prefix, files, 0); + listObjects(path_prefix, files, max_keys); return std::make_shared(std::move(files)); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index eae31af9d44..d4ac6ea0239 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -37,6 +37,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } class ReadBufferFromFileBase; @@ -47,21 +48,28 @@ using ObjectAttributes = std::map; struct ObjectMetadata { uint64_t size_bytes = 0; - std::optional last_modified; - std::optional attributes; + Poco::Timestamp last_modified; + ObjectAttributes attributes; }; struct RelativePathWithMetadata { String relative_path; - ObjectMetadata metadata; + std::optional metadata; RelativePathWithMetadata() = default; - RelativePathWithMetadata(String relative_path_, ObjectMetadata metadata_) + explicit RelativePathWithMetadata(String relative_path_, std::optional metadata_ = std::nullopt) : relative_path(std::move(relative_path_)) , metadata(std::move(metadata_)) {} + + virtual ~RelativePathWithMetadata() = default; + + virtual std::string getFileName() const { return std::filesystem::path(relative_path).filename(); } + virtual std::string getPath() const { return relative_path; } + virtual bool isArchive() const { return false; } + virtual std::string getPathToArchive() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not an archive"); } }; struct ObjectKeyWithMetadata @@ -77,7 +85,8 @@ struct ObjectKeyWithMetadata {} }; -using RelativePathsWithMetadata = std::vector; +using RelativePathWithMetadataPtr = std::shared_ptr; +using RelativePathsWithMetadata = std::vector; using ObjectKeysWithMetadata = std::vector; class IObjectStorageIterator; @@ -111,9 +120,9 @@ public: /// /, /a, /a/b, /a/b/c, /a/b/c/d while exists will return true only for /a/b/c/d virtual bool existsOrHasAnyChild(const std::string & path) const; - virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const; + virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const; - virtual ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const; + virtual ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const; /// Get object metadata if supported. It should be possible to receive /// at least size of object @@ -190,11 +199,15 @@ public: virtual void startup() = 0; /// Apply new settings, in most cases reiniatilize client and some other staff + struct ApplyNewSettingsOptions + { + bool allow_client_change = true; + }; virtual void applyNewSettings( - const Poco::Util::AbstractConfiguration &, + const Poco::Util::AbstractConfiguration & /* config */, const std::string & /*config_prefix*/, - ContextPtr) - {} + ContextPtr /* context */, + const ApplyNewSettingsOptions & /* options */) {} /// Sometimes object storages have something similar to chroot or namespace, for example /// buckets in S3. If object storage doesn't have any namepaces return empty string. diff --git a/src/Disks/ObjectStorages/IObjectStorage_fwd.h b/src/Disks/ObjectStorages/IObjectStorage_fwd.h index f6ebc883682..67efa4aae2b 100644 --- a/src/Disks/ObjectStorages/IObjectStorage_fwd.h +++ b/src/Disks/ObjectStorages/IObjectStorage_fwd.h @@ -10,4 +10,7 @@ using ObjectStoragePtr = std::shared_ptr; class IMetadataStorage; using MetadataStoragePtr = std::shared_ptr; +class IObjectStorageIterator; +using ObjectStorageIteratorPtr = std::shared_ptr; + } diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index d44e17a0713..a247d86ddce 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -172,7 +172,7 @@ ObjectMetadata LocalObjectStorage::getObjectMetadata(const std::string & path) c return object_metadata; } -void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int /* max_keys */) const +void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t/* max_keys */) const { for (const auto & entry : fs::directory_iterator(path)) { @@ -182,8 +182,7 @@ void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWith continue; } - auto metadata = getObjectMetadata(entry.path()); - children.emplace_back(entry.path(), std::move(metadata)); + children.emplace_back(std::make_shared(entry.path(), getObjectMetadata(entry.path()))); } } @@ -223,11 +222,6 @@ std::unique_ptr LocalObjectStorage::cloneObjectStorage( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "cloneObjectStorage() is not implemented for LocalObjectStorage"); } -void LocalObjectStorage::applyNewSettings( - const Poco::Util::AbstractConfiguration & /* config */, const std::string & /* config_prefix */, ContextPtr /* context */) -{ -} - ObjectStorageKey LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const { constexpr size_t key_name_total_size = 32; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index 22429a99c76..371cd37f8b2 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -58,7 +58,7 @@ public: ObjectMetadata getObjectMetadata(const std::string & path) const override; - void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; bool existsOrHasAnyChild(const std::string & path) const override; @@ -73,11 +73,6 @@ public: void startup() override; - void applyNewSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - String getObjectsNamespace() const override { return ""; } std::unique_ptr cloneObjectStorage( diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index faa7ca38b75..30111d04d20 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -105,7 +105,7 @@ std::vector MetadataStorageFromPlainObjectStorage::getDirectChildre std::unordered_set duplicates_filter; for (const auto & elem : remote_paths) { - const auto & path = elem.relative_path; + const auto & path = elem->relative_path; chassert(path.find(storage_key) == 0); const auto child_pos = storage_key.size(); /// string::npos is ok. diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp index d910dae80b4..3e772271b99 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.cpp @@ -26,11 +26,11 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri object_storage->listObjects(root, files, 0); for (const auto & file : files) { - auto remote_path = std::filesystem::path(file.relative_path); + auto remote_path = std::filesystem::path(file->relative_path); if (remote_path.filename() != PREFIX_PATH_FILE_NAME) continue; - StoredObject object{file.relative_path}; + StoredObject object{file->relative_path}; auto read_buf = object_storage->readObject(object); String local_path; @@ -88,7 +88,7 @@ std::vector getDirectChildrenOnRewritableDisk( auto skip_list = std::set{PREFIX_PATH_FILE_NAME}; for (const auto & elem : remote_paths) { - const auto & path = elem.relative_path; + const auto & path = elem->relative_path; chassert(path.find(storage_key) == 0); const auto child_pos = storage_key.size(); diff --git a/src/Disks/ObjectStorages/MetadataStorageTransactionState.cpp b/src/Disks/ObjectStorages/MetadataStorageTransactionState.cpp index 245578b5d9e..a37f4ce7e65 100644 --- a/src/Disks/ObjectStorages/MetadataStorageTransactionState.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageTransactionState.cpp @@ -17,7 +17,6 @@ std::string toString(MetadataStorageTransactionState state) case MetadataStorageTransactionState::PARTIALLY_ROLLED_BACK: return "PARTIALLY_ROLLED_BACK"; } - UNREACHABLE(); } } diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index c83b9247b99..d7884c2911b 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -9,7 +9,7 @@ #endif #if USE_HDFS && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) #include -#include +#include #endif #if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) #include @@ -183,7 +183,7 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings); + auto client = getClient(config, config_prefix, context, *settings, true); auto key_generator = getKeyGenerator(uri, config, config_prefix); auto object_storage = createObjectStorage( @@ -219,7 +219,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings); + auto client = getClient(config, config_prefix, context, *settings, true); auto key_generator = getKeyGenerator(uri, config, config_prefix); auto object_storage = std::make_shared>( @@ -253,7 +253,7 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings); + auto client = getClient(config, config_prefix, context, *settings, true); auto key_generator = getKeyGenerator(uri, config, config_prefix); auto object_storage = std::make_shared>( @@ -287,10 +287,9 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory) std::unique_ptr settings = std::make_unique( config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), context->getSettingsRef().hdfs_replication); - return createObjectStorage(ObjectStorageType::HDFS, config, config_prefix, uri, std::move(settings), config); + return createObjectStorage(ObjectStorageType::HDFS, config, config_prefix, uri, std::move(settings), config, /* lazy_initialize */false); }); } #endif diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.cpp b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp index 72ec6e0e500..3d939ce9230 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp @@ -9,7 +9,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -RelativePathWithMetadata ObjectStorageIteratorFromList::current() +RelativePathWithMetadataPtr ObjectStorageIteratorFromList::current() { if (!isValid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.h b/src/Disks/ObjectStorages/ObjectStorageIterator.h index 9af2593579a..26c3c690ba5 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.h +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.h @@ -12,9 +12,9 @@ public: virtual void next() = 0; virtual void nextBatch() = 0; virtual bool isValid() = 0; - virtual RelativePathWithMetadata current() = 0; + virtual RelativePathWithMetadataPtr current() = 0; virtual RelativePathsWithMetadata currentBatch() = 0; - virtual std::optional getCurrrentBatchAndScheduleNext() = 0; + virtual std::optional getCurrentBatchAndScheduleNext() = 0; virtual size_t getAccumulatedSize() const = 0; virtual ~IObjectStorageIterator() = default; @@ -27,9 +27,7 @@ class ObjectStorageIteratorFromList : public IObjectStorageIterator public: explicit ObjectStorageIteratorFromList(RelativePathsWithMetadata && batch_) : batch(std::move(batch_)) - , batch_iterator(batch.begin()) - { - } + , batch_iterator(batch.begin()) {} void next() override { @@ -37,32 +35,26 @@ public: ++batch_iterator; } - void nextBatch() override + void nextBatch() override { batch_iterator = batch.end(); } + + bool isValid() override { return batch_iterator != batch.end(); } + + RelativePathWithMetadataPtr current() override; + + RelativePathsWithMetadata currentBatch() override { return batch; } + + std::optional getCurrentBatchAndScheduleNext() override { - batch_iterator = batch.end(); + if (batch.empty()) + return {}; + + auto current_batch = std::move(batch); + batch = {}; + return current_batch; } - bool isValid() override - { - return batch_iterator != batch.end(); - } + size_t getAccumulatedSize() const override { return batch.size(); } - RelativePathWithMetadata current() override; - - RelativePathsWithMetadata currentBatch() override - { - return batch; - } - - std::optional getCurrrentBatchAndScheduleNext() override - { - return std::nullopt; - } - - size_t getAccumulatedSize() const override - { - return batch.size(); - } private: RelativePathsWithMetadata batch; RelativePathsWithMetadata::iterator batch_iterator; diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index 990e66fc4e5..0420de0f8dd 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -11,10 +11,37 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +IObjectStorageIteratorAsync::IObjectStorageIteratorAsync( + CurrentMetrics::Metric threads_metric, + CurrentMetrics::Metric threads_active_metric, + CurrentMetrics::Metric threads_scheduled_metric, + const std::string & thread_name) + : list_objects_pool(threads_metric, threads_active_metric, threads_scheduled_metric, 1) + , list_objects_scheduler(threadPoolCallbackRunnerUnsafe(list_objects_pool, thread_name)) +{ +} + +IObjectStorageIteratorAsync::~IObjectStorageIteratorAsync() +{ + if (!deactivated) + deactivate(); +} + +void IObjectStorageIteratorAsync::deactivate() +{ + list_objects_pool.wait(); + deactivated = true; +} + void IObjectStorageIteratorAsync::nextBatch() { std::lock_guard lock(mutex); - if (!is_finished) + if (is_finished) + { + current_batch.clear(); + current_batch_iterator = current_batch.begin(); + } + else { if (!is_initialized) { @@ -22,19 +49,27 @@ void IObjectStorageIteratorAsync::nextBatch() is_initialized = true; } - BatchAndHasNext next_batch = outcome_future.get(); - current_batch = std::move(next_batch.batch); - accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); - current_batch_iterator = current_batch.begin(); - if (next_batch.has_next) - outcome_future = scheduleBatch(); - else - is_finished = true; - } - else - { - current_batch.clear(); + chassert(outcome_future.valid()); + BatchAndHasNext result; + try + { + result = outcome_future.get(); + } + catch (...) + { + is_finished = true; + throw; + } + + current_batch = std::move(result.batch); current_batch_iterator = current_batch.begin(); + + accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); + + if (result.has_next) + outcome_future = scheduleBatch(); + else + is_finished = true; } } @@ -42,24 +77,10 @@ void IObjectStorageIteratorAsync::next() { std::lock_guard lock(mutex); - if (current_batch_iterator != current_batch.end()) - { + if (current_batch_iterator == current_batch.end()) + nextBatch(); + else ++current_batch_iterator; - } - else if (!is_finished) - { - if (outcome_future.valid()) - { - BatchAndHasNext next_batch = outcome_future.get(); - current_batch = std::move(next_batch.batch); - accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); - current_batch_iterator = current_batch.begin(); - if (next_batch.has_next) - outcome_future = scheduleBatch(); - else - is_finished = true; - } - } } std::future IObjectStorageIteratorAsync::scheduleBatch() @@ -72,7 +93,6 @@ std::future IObjectStorageIterator }, Priority{}); } - bool IObjectStorageIteratorAsync::isValid() { if (!is_initialized) @@ -82,7 +102,7 @@ bool IObjectStorageIteratorAsync::isValid() return current_batch_iterator != current_batch.end(); } -RelativePathWithMetadata IObjectStorageIteratorAsync::current() +RelativePathWithMetadataPtr IObjectStorageIteratorAsync::current() { if (!isValid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); @@ -101,20 +121,20 @@ RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch() return current_batch; } -std::optional IObjectStorageIteratorAsync::getCurrrentBatchAndScheduleNext() +std::optional IObjectStorageIteratorAsync::getCurrentBatchAndScheduleNext() { std::lock_guard lock(mutex); if (!is_initialized) nextBatch(); - if (current_batch_iterator != current_batch.end()) + if (current_batch_iterator == current_batch.end()) { - auto temp_current_batch = current_batch; - nextBatch(); - return temp_current_batch; + return std::nullopt; } - return std::nullopt; + auto temp_current_batch = std::move(current_batch); + nextBatch(); + return temp_current_batch; } size_t IObjectStorageIteratorAsync::getAccumulatedSize() const diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index 7fdb02bdfe2..cb4818d01ae 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -17,24 +17,22 @@ public: CurrentMetrics::Metric threads_metric, CurrentMetrics::Metric threads_active_metric, CurrentMetrics::Metric threads_scheduled_metric, - const std::string & thread_name) - : list_objects_pool(threads_metric, threads_active_metric, threads_scheduled_metric, 1) - , list_objects_scheduler(threadPoolCallbackRunnerUnsafe(list_objects_pool, thread_name)) - { - } + const std::string & thread_name); + + ~IObjectStorageIteratorAsync() override; + + bool isValid() override; + + RelativePathWithMetadataPtr current() override; + RelativePathsWithMetadata currentBatch() override; void next() override; void nextBatch() override; - bool isValid() override; - RelativePathWithMetadata current() override; - RelativePathsWithMetadata currentBatch() override; - size_t getAccumulatedSize() const override; - std::optional getCurrrentBatchAndScheduleNext() override; - ~IObjectStorageIteratorAsync() override - { - list_objects_pool.wait(); - } + size_t getAccumulatedSize() const override; + std::optional getCurrentBatchAndScheduleNext() override; + + void deactivate(); protected: @@ -50,6 +48,7 @@ protected: bool is_initialized{false}; bool is_finished{false}; + bool deactivated{false}; mutable std::recursive_mutex mutex; ThreadPool list_objects_pool; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index adbdd9d13aa..69485bd4d01 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -61,7 +61,10 @@ void throwIfError(const Aws::Utils::Outcome & response) if (!response.IsSuccess()) { const auto & err = response.GetError(); - throw S3Exception(fmt::format("{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())), err.GetErrorType()); + throw S3Exception( + fmt::format("{} (Code: {}, s3 exception: {})", + err.GetMessage(), static_cast(err.GetErrorType()), err.GetExceptionName()), + err.GetErrorType()); } } @@ -111,10 +114,19 @@ public: CurrentMetrics::ObjectStorageS3ThreadsScheduled, "ListObjectS3") , client(client_) + , request(std::make_unique()) { - request.SetBucket(bucket_); - request.SetPrefix(path_prefix); - request.SetMaxKeys(static_cast(max_list_size)); + request->SetBucket(bucket_); + request->SetPrefix(path_prefix); + request->SetMaxKeys(static_cast(max_list_size)); + } + + ~S3IteratorAsync() override + { + /// Deactivate background threads before resetting the request to avoid data race. + deactivate(); + request.reset(); + client.reset(); } private: @@ -123,34 +135,32 @@ private: ProfileEvents::increment(ProfileEvents::S3ListObjects); ProfileEvents::increment(ProfileEvents::DiskS3ListObjects); - bool result = false; - auto outcome = client->ListObjectsV2(request); + auto outcome = client->ListObjectsV2(*request); + /// Outcome failure will be handled on the caller side. if (outcome.IsSuccess()) { + request->SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + auto objects = outcome.GetResult().GetContents(); - - result = !objects.empty(); - for (const auto & object : objects) - batch.emplace_back( - object.GetKey(), - ObjectMetadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}} - ); + { + ObjectMetadata metadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}}; + batch.emplace_back(std::make_shared(object.GetKey(), std::move(metadata))); + } - if (result) - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - - return result; + /// It returns false when all objects were returned + return outcome.GetResult().GetIsTruncated(); } - throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", - quoteString(request.GetBucket()), quoteString(request.GetPrefix()), - backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); + throw S3Exception(outcome.GetError().GetErrorType(), + "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + quoteString(request->GetBucket()), quoteString(request->GetPrefix()), + backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); } std::shared_ptr client; - S3::ListObjectsV2Request request; + std::unique_ptr request; }; } @@ -248,12 +258,16 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (mode != WriteMode::Rewrite) throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); - auto settings_ptr = s3_settings.get(); + S3Settings::RequestSettings request_settings = s3_settings.get()->request_settings; + if (auto query_context = CurrentThread::getQueryContext()) + { + request_settings.updateFromSettingsIfChanged(query_context->getSettingsRef()); + } + ThreadPoolCallbackRunnerUnsafe scheduler; if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunnerUnsafe(getThreadPoolWriter(), "VFSWrite"); - auto blob_storage_log = BlobStorageLogWriter::create(disk_name); if (blob_storage_log) blob_storage_log->local_path = object.local_path; @@ -263,7 +277,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN uri.bucket, object.remote_path, buf_size, - settings_ptr->request_settings, + request_settings, std::move(blob_storage_log), attributes, std::move(scheduler), @@ -271,13 +285,13 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN } -ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix) const +ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix, size_t max_keys) const { auto settings_ptr = s3_settings.get(); - return std::make_shared(uri.bucket, path_prefix, client.get(), settings_ptr->list_object_keys_size); + return std::make_shared(uri.bucket, path_prefix, client.get(), max_keys); } -void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { auto settings_ptr = s3_settings.get(); @@ -285,7 +299,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet request.SetBucket(uri.bucket); request.SetPrefix(path); if (max_keys) - request.SetMaxKeys(max_keys); + request.SetMaxKeys(static_cast(max_keys)); else request.SetMaxKeys(settings_ptr->list_object_keys_size); @@ -305,19 +319,19 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet break; for (const auto & object : objects) - children.emplace_back( + children.emplace_back(std::make_shared( object.GetKey(), ObjectMetadata{ static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), - {}}); + {}})); if (max_keys) { - int keys_left = max_keys - static_cast(children.size()); + size_t keys_left = max_keys - children.size(); if (keys_left <= 0) break; - request.SetMaxKeys(keys_left); + request.SetMaxKeys(static_cast(keys_left)); } request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); @@ -425,14 +439,16 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* throw_on_error= */ false); + auto object_info = S3::getObjectInfo( + *client.get(), uri.bucket, path, {}, settings_ptr->request_settings, + /* with_metadata= */ true, /* throw_on_error= */ false); if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) return {}; ObjectMetadata result; result.size_bytes = object_info.size; - result.last_modified = object_info.last_modification_time; + result.last_modified = Poco::Timestamp::fromEpochTime(object_info.last_modification_time); result.attributes = object_info.metadata; return result; @@ -441,11 +457,20 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true); + S3::ObjectInfo object_info; + try + { + object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true); + } + catch (DB::Exception & e) + { + e.addMessage("while reading " + path); + throw; + } ObjectMetadata result; result.size_bytes = object_info.size; - result.last_modified = object_info.last_modification_time; + result.last_modified = Poco::Timestamp::fromEpochTime(object_info.last_modification_time); result.attributes = object_info.metadata; return result; @@ -544,19 +569,37 @@ void S3ObjectStorage::startup() const_cast(*client.get()).EnableRequestProcessing(); } -void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) +void S3ObjectStorage::applyNewSettings( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + ContextPtr context, + const ApplyNewSettingsOptions & options) { - auto new_s3_settings = getSettings(config, config_prefix, context); - auto new_client = getClient(config, config_prefix, context, *new_s3_settings); - s3_settings.set(std::move(new_s3_settings)); - client.set(std::move(new_client)); + auto settings_from_config = getSettings(config, config_prefix, context, context->getSettingsRef().s3_validate_request_settings); + auto modified_settings = std::make_unique(*s3_settings.get()); + modified_settings->auth_settings.updateFrom(settings_from_config->auth_settings); + + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString(), context->getUserName())) + modified_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); + + auto current_settings = s3_settings.get(); + if (options.allow_client_change + && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3)) + { + auto new_client = getClient(config, config_prefix, context, *modified_settings, for_disk_s3, &uri); + client.set(std::move(new_client)); + } + s3_settings.set(std::move(modified_settings)); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( - const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) + const std::string & new_namespace, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + ContextPtr context) { auto new_s3_settings = getSettings(config, config_prefix, context); - auto new_client = getClient(config, config_prefix, context, *new_s3_settings); + auto new_client = getClient(config, config_prefix, context, *new_s3_settings, true); auto new_uri{uri}; new_uri.bucket = new_namespace; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 5eaab4b585c..062ddd4e2a2 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -21,11 +21,13 @@ struct S3ObjectStorageSettings S3ObjectStorageSettings( const S3Settings::RequestSettings & request_settings_, + const S3::AuthSettings & auth_settings_, uint64_t min_bytes_for_seek_, int32_t list_object_keys_size_, int32_t objects_chunk_size_to_delete_, bool read_only_) : request_settings(request_settings_) + , auth_settings(auth_settings_) , min_bytes_for_seek(min_bytes_for_seek_) , list_object_keys_size(list_object_keys_size_) , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) @@ -33,6 +35,7 @@ struct S3ObjectStorageSettings {} S3Settings::RequestSettings request_settings; + S3::AuthSettings auth_settings; uint64_t min_bytes_for_seek; int32_t list_object_keys_size; @@ -50,7 +53,8 @@ private: S3::URI uri_, const S3Capabilities & s3_capabilities_, ObjectStorageKeysGeneratorPtr key_generator_, - const String & disk_name_) + const String & disk_name_, + bool for_disk_s3_ = true) : uri(uri_) , disk_name(disk_name_) , client(std::move(client_)) @@ -58,6 +62,7 @@ private: , s3_capabilities(s3_capabilities_) , key_generator(std::move(key_generator_)) , log(getLogger(logger_name)) + , for_disk_s3(for_disk_s3_) { } @@ -98,9 +103,9 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; - ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const override; + ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const override; /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; @@ -142,7 +147,8 @@ public: void applyNewSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context) override; + ContextPtr context, + const ApplyNewSettingsOptions & options) override; std::string getObjectsNamespace() const override { return uri.bucket; } @@ -179,6 +185,8 @@ private: ObjectStorageKeysGeneratorPtr key_generator; LoggerPtr log; + + const bool for_disk_s3; }; } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 35913613326..139472a8b01 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -25,19 +25,29 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} namespace ErrorCodes { extern const int NO_ELEMENTS_IN_CONFIG; } -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) +std::unique_ptr getSettings( + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + ContextPtr context, + bool validate_settings) { const Settings & settings = context->getSettingsRef(); - S3Settings::RequestSettings request_settings(config, config_prefix, settings, "s3_"); + auto request_settings = S3Settings::RequestSettings(config, config_prefix, settings, "s3_", validate_settings); + auto auth_settings = S3::AuthSettings::loadFromConfig(config_prefix, config); return std::make_unique( request_settings, + auth_settings, config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".list_object_keys_size", 1000), config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), @@ -48,82 +58,99 @@ std::unique_ptr getClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, - const S3ObjectStorageSettings & settings) + const S3ObjectStorageSettings & settings, + bool for_disk_s3, + const S3::URI * url_) { const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); const Settings & local_settings = context->getSettingsRef(); - const String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); - S3::URI uri(endpoint); - if (!uri.key.ends_with('/')) - uri.key.push_back('/'); + const auto & auth_settings = settings.auth_settings; + const auto & request_settings = settings.request_settings; - if (S3::isS3ExpressEndpoint(endpoint) && !config.has(config_prefix + ".region")) + S3::URI url; + if (for_disk_s3) + { + String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); + url = S3::URI(endpoint); + if (!url.key.ends_with('/')) + url.key.push_back('/'); + } + else + { + if (!url_) + throw Exception(ErrorCodes::LOGICAL_ERROR, "URL not passed"); + url = *url_; + } + const bool is_s3_express_bucket = S3::isS3ExpressEndpoint(url.endpoint); + if (is_s3_express_bucket && !config.has(config_prefix + ".region")) + { throw Exception( ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Region should be explicitly specified for directory buckets ({})", config_prefix); + } S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - config.getString(config_prefix + ".region", ""), + auth_settings.region, context->getRemoteHostFilter(), static_cast(global_settings.s3_max_redirects), static_cast(global_settings.s3_retry_attempts), global_settings.enable_s3_requests_logging, - /* for_disk_s3 = */ true, + for_disk_s3, settings.request_settings.get_request_throttler, settings.request_settings.put_request_throttler, - uri.uri.getScheme()); + url.uri.getScheme()); - client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", S3::DEFAULT_CONNECT_TIMEOUT_MS); - client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS); - client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS); + client_configuration.connectTimeoutMs = config.getUInt64(config_prefix + ".connect_timeout_ms", local_settings.s3_connect_timeout_ms.value); + client_configuration.requestTimeoutMs = config.getUInt64(config_prefix + ".request_timeout_ms", local_settings.s3_request_timeout_ms.value); + client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", static_cast(request_settings.max_connections)); client_configuration.http_keep_alive_timeout = config.getUInt(config_prefix + ".http_keep_alive_timeout", S3::DEFAULT_KEEP_ALIVE_TIMEOUT); client_configuration.http_keep_alive_max_requests = config.getUInt(config_prefix + ".http_keep_alive_max_requests", S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS); - client_configuration.endpointOverride = uri.endpoint; + client_configuration.endpointOverride = url.endpoint; client_configuration.s3_use_adaptive_timeouts = config.getBool( config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); - /* - * Override proxy configuration for backwards compatibility with old configuration format. - * */ - auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( - ProxyConfiguration::protocolFromString(uri.uri.getScheme()), - config_prefix, - config - ); - if (proxy_config) + if (for_disk_s3) { - client_configuration.per_request_configuration - = [proxy_config]() { return proxy_config->resolve(); }; - client_configuration.error_report - = [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); }; + /* + * Override proxy configuration for backwards compatibility with old configuration format. + * */ + if (auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + ProxyConfiguration::protocolFromString(url.uri.getScheme()), config_prefix, config)) + { + client_configuration.per_request_configuration + = [proxy_config]() { return proxy_config->resolve(); }; + client_configuration.error_report + = [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); }; + } } - HTTPHeaderEntries headers = S3::getHTTPHeaders(config_prefix, config); S3::ServerSideEncryptionKMSConfig sse_kms_config = S3::getSSEKMSConfig(config_prefix, config); - S3::ClientSettings client_settings{ - .use_virtual_addressing = uri.is_virtual_hosted_style, + .use_virtual_addressing = url.is_virtual_hosted_style, .disable_checksum = local_settings.s3_disable_checksum, .gcs_issue_compose_request = config.getBool("s3.gcs_issue_compose_request", false), - .is_s3express_bucket = S3::isS3ExpressEndpoint(endpoint), + .is_s3express_bucket = is_s3_express_bucket, + }; + + auto credentials_configuration = S3::CredentialsConfiguration + { + auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), + auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + auth_settings.expiration_window_seconds.value_or(context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), + auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), }; return S3::ClientFactory::instance().create( client_configuration, client_settings, - config.getString(config_prefix + ".access_key_id", ""), - config.getString(config_prefix + ".secret_access_key", ""), - config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""), + auth_settings.access_key_id, + auth_settings.secret_access_key, + auth_settings.server_side_encryption_customer_key_base64, std::move(sse_kms_config), - std::move(headers), - S3::CredentialsConfiguration - { - config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", true)), - config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false)), - config.getUInt64(config_prefix + ".expiration_window_seconds", config.getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), - config.getBool(config_prefix + ".no_sign_request", config.getBool("s3.no_sign_request", false)) - }); + auth_settings.headers, + credentials_configuration, + auth_settings.session_token); } } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index e461daa99e2..11ac64ce913 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -14,9 +14,19 @@ namespace DB struct S3ObjectStorageSettings; -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); +std::unique_ptr getSettings( + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + ContextPtr context, + bool validate_settings = true); -std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, const S3ObjectStorageSettings & settings); +std::unique_ptr getClient( + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + ContextPtr context, + const S3ObjectStorageSettings & settings, + bool for_disk_s3, + const S3::URI * url_ = nullptr); } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 69f6137cd2d..e837e056acc 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -344,11 +344,6 @@ void WebObjectStorage::startup() { } -void WebObjectStorage::applyNewSettings( - const Poco::Util::AbstractConfiguration & /* config */, const std::string & /* config_prefix */, ContextPtr /* context */) -{ -} - ObjectMetadata WebObjectStorage::getObjectMetadata(const std::string & /* path */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Metadata is not supported for {}", getName()); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index b8ab510a6fb..9d3b9a3a8f0 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -72,11 +72,6 @@ public: void startup() override; - void applyNewSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - String getObjectsNamespace() const override { return ""; } std::unique_ptr cloneObjectStorage( diff --git a/src/Disks/VolumeJBOD.cpp b/src/Disks/VolumeJBOD.cpp index d0e9d32ff5e..f8b9a57affe 100644 --- a/src/Disks/VolumeJBOD.cpp +++ b/src/Disks/VolumeJBOD.cpp @@ -112,7 +112,6 @@ DiskPtr VolumeJBOD::getDisk(size_t /* index */) const return disks_by_size.top().disk; } } - UNREACHABLE(); } ReservationPtr VolumeJBOD::reserve(UInt64 bytes) @@ -164,7 +163,6 @@ ReservationPtr VolumeJBOD::reserve(UInt64 bytes) return reservation; } } - UNREACHABLE(); } bool VolumeJBOD::areMergesAvoided() const diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 89a7a31d033..9577ca2a8df 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -62,7 +62,6 @@ String escapingRuleToString(FormatSettings::EscapingRule escaping_rule) case FormatSettings::EscapingRule::Raw: return "Raw"; } - UNREACHABLE(); } void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index 3b057779ffe..d85bb0e7060 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -21,8 +21,6 @@ namespace ErrorCodes const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column) { - if (!column) - return {}; if (!isColumnConst(*column)) return {}; diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 99f3a14dfec..dde57e8320d 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -149,8 +149,6 @@ struct IntegerRoundingComputation return x; } } - - UNREACHABLE(); } static ALWAYS_INLINE T compute(T x, T scale) @@ -163,8 +161,6 @@ struct IntegerRoundingComputation case ScaleMode::Negative: return computeImpl(x, scale); } - - UNREACHABLE(); } static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) requires std::integral @@ -247,8 +243,6 @@ inline float roundWithMode(float x, RoundingMode mode) case RoundingMode::Ceil: return ceilf(x); case RoundingMode::Trunc: return truncf(x); } - - UNREACHABLE(); } inline double roundWithMode(double x, RoundingMode mode) @@ -260,8 +254,6 @@ inline double roundWithMode(double x, RoundingMode mode) case RoundingMode::Ceil: return ceil(x); case RoundingMode::Trunc: return trunc(x); } - - UNREACHABLE(); } template diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index 1c9f28c9724..f93a885ee65 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -232,7 +232,6 @@ struct TimeWindowImpl default: throw Exception(ErrorCodes::SYNTAX_ERROR, "Fraction seconds are unsupported by windows yet"); } - UNREACHABLE(); } template @@ -422,7 +421,6 @@ struct TimeWindowImpl default: throw Exception(ErrorCodes::SYNTAX_ERROR, "Fraction seconds are unsupported by windows yet"); } - UNREACHABLE(); } template diff --git a/src/Functions/PolygonUtils.h b/src/Functions/PolygonUtils.h index c4851718da6..57f1243537d 100644 --- a/src/Functions/PolygonUtils.h +++ b/src/Functions/PolygonUtils.h @@ -381,8 +381,6 @@ bool PointInPolygonWithGrid::contains(CoordinateType x, Coordina case CellType::complexPolygon: return boost::geometry::within(Point(x, y), polygons[cell.index_of_inner_polygon]); } - - UNREACHABLE(); } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp index 568e0b9b5d2..766d63eafb0 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp @@ -35,7 +35,6 @@ namespace case UserDefinedSQLObjectType::Function: return "function_"; } - UNREACHABLE(); } constexpr std::string_view sql_extension = ".sql"; diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp new file mode 100644 index 00000000000..c3f7701a05a --- /dev/null +++ b/src/Functions/generateSnowflakeID.cpp @@ -0,0 +1,255 @@ +#include +#include +#include +#include +#include +#include +#include +#include "base/types.h" + + +namespace DB +{ + +namespace +{ + +/* Snowflake ID + https://en.wikipedia.org/wiki/Snowflake_ID + + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|0| timestamp | +├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| | machine_id | machine_seq_num | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ + +- The first 41 (+ 1 top zero bit) bits is the timestamp (millisecond since Unix epoch 1 Jan 1970) +- The middle 10 bits are the machine ID +- The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by different processes +*/ + +/// bit counts +constexpr auto timestamp_bits_count = 41; +constexpr auto machine_id_bits_count = 10; +constexpr auto machine_seq_num_bits_count = 12; + +/// bits masks for Snowflake ID components +constexpr uint64_t machine_id_mask = ((1ull << machine_id_bits_count) - 1) << machine_seq_num_bits_count; +constexpr uint64_t machine_seq_num_mask = (1ull << machine_seq_num_bits_count) - 1; + +/// max values +constexpr uint64_t max_machine_seq_num = machine_seq_num_mask; + +uint64_t getTimestamp() +{ + auto now = std::chrono::system_clock::now(); + auto ticks_since_epoch = std::chrono::duration_cast(now.time_since_epoch()).count(); + return static_cast(ticks_since_epoch) & ((1ull << timestamp_bits_count) - 1); +} + +uint64_t getMachineIdImpl() +{ + UUID server_uuid = ServerUUID::get(); + /// hash into 64 bits + uint64_t hi = UUIDHelpers::getHighBytes(server_uuid); + uint64_t lo = UUIDHelpers::getLowBytes(server_uuid); + /// return only 10 bits + return (((hi * 11) ^ (lo * 17)) & machine_id_mask) >> machine_seq_num_bits_count; +} + +uint64_t getMachineId() +{ + static uint64_t machine_id = getMachineIdImpl(); + return machine_id; +} + +struct SnowflakeId +{ + uint64_t timestamp; + uint64_t machine_id; + uint64_t machine_seq_num; +}; + +SnowflakeId toSnowflakeId(uint64_t snowflake) +{ + return {.timestamp = (snowflake >> (machine_id_bits_count + machine_seq_num_bits_count)), + .machine_id = ((snowflake & machine_id_mask) >> machine_seq_num_bits_count), + .machine_seq_num = (snowflake & machine_seq_num_mask)}; +} + +uint64_t fromSnowflakeId(SnowflakeId components) +{ + return (components.timestamp << (machine_id_bits_count + machine_seq_num_bits_count) | + components.machine_id << (machine_seq_num_bits_count) | + components.machine_seq_num); +} + +struct SnowflakeIdRange +{ + SnowflakeId begin; /// inclusive + SnowflakeId end; /// exclusive +}; + +/// To get the range of `input_rows_count` Snowflake IDs from `max(available, now)`: +/// 1. calculate Snowflake ID by current timestamp (`now`) +/// 2. `begin = max(available, now)` +/// 3. Calculate `end = begin + input_rows_count` handling `machine_seq_num` overflow +SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count) +{ + /// 1. `now` + SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = getMachineId(), .machine_seq_num = 0}; + + /// 2. `begin` + if (begin.timestamp <= available.timestamp) + { + begin.timestamp = available.timestamp; + begin.machine_seq_num = available.machine_seq_num; + } + + /// 3. `end = begin + input_rows_count` + SnowflakeId end; + const uint64_t seq_nums_in_current_timestamp_left = (max_machine_seq_num - begin.machine_seq_num + 1); + if (input_rows_count >= seq_nums_in_current_timestamp_left) + /// if sequence numbers in current timestamp is not enough for rows --> depending on how many elements input_rows_count overflows, forward timestamp by at least 1 tick + end.timestamp = begin.timestamp + 1 + (input_rows_count - seq_nums_in_current_timestamp_left) / (max_machine_seq_num + 1); + else + end.timestamp = begin.timestamp; + + end.machine_id = begin.machine_id; + end.machine_seq_num = (begin.machine_seq_num + input_rows_count) & machine_seq_num_mask; + + return {begin, end}; +} + +struct GlobalCounterPolicy +{ + static constexpr auto name = "generateSnowflakeID"; + static constexpr auto description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; + + /// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously. + struct Data + { + static inline std::atomic lowest_available_snowflake_id = 0; + + SnowflakeId reserveRange(size_t input_rows_count) + { + uint64_t available_snowflake_id = lowest_available_snowflake_id.load(); + SnowflakeIdRange range; + do + { + range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count); + } + while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end))); + /// if CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try + /// else --> our thread reserved ID range [begin, end) and return the beginning of the range + + return range.begin; + } + }; +}; + +struct ThreadLocalCounterPolicy +{ + static constexpr auto name = "generateSnowflakeIDThreadMonotonic"; + static constexpr auto description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. This function behaves like generateSnowflakeID but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs.)"; + + /// Guarantee counter monotonicity within one timestamp within the same thread. Faster than GlobalCounterPolicy if a query uses multiple threads. + struct Data + { + static inline thread_local uint64_t lowest_available_snowflake_id = 0; + + SnowflakeId reserveRange(size_t input_rows_count) + { + SnowflakeIdRange range = getRangeOfAvailableIds(toSnowflakeId(lowest_available_snowflake_id), input_rows_count); + lowest_available_snowflake_id = fromSnowflakeId(range.end); + return range.begin; + } + }; +}; + +} + +template +class FunctionGenerateSnowflakeID : public IFunction, public FillPolicy +{ +public: + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared(); } + + String getName() const override { return FillPolicy::name; } + size_t getNumberOfArguments() const override { return 0; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors mandatory_args; + FunctionArgumentDescriptors optional_args{ + {"expr", nullptr, nullptr, "Arbitrary expression"} + }; + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr &, size_t input_rows_count) const override + { + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_res->getData(); + + if (input_rows_count != 0) + { + vec_to.resize(input_rows_count); + + typename FillPolicy::Data data; + SnowflakeId snowflake_id = data.reserveRange(input_rows_count); /// returns begin of available snowflake ids range + + for (UInt64 & to_row : vec_to) + { + to_row = fromSnowflakeId(snowflake_id); + if (snowflake_id.machine_seq_num == max_machine_seq_num) + { + /// handle overflow + snowflake_id.machine_seq_num = 0; + ++snowflake_id.timestamp; + } + else + { + ++snowflake_id.machine_seq_num; + } + } + } + + return col_res; + } + +}; + +template +void registerSnowflakeIDGenerator(auto & factory) +{ + static constexpr auto doc_syntax_format = "{}([expression])"; + static constexpr auto example_format = "SELECT {}()"; + static constexpr auto multiple_example_format = "SELECT {f}(1), {f}(2)"; + + FunctionDocumentation::Description description = FillPolicy::description; + FunctionDocumentation::Syntax syntax = fmt::format(doc_syntax_format, FillPolicy::name); + FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; + FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64"; + FunctionDocumentation::Examples examples = {{"single", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; + FunctionDocumentation::Categories categories = {"Snowflake ID"}; + + factory.template registerFunction>({description, syntax, arguments, returned_value, examples, categories}, FunctionFactory::CaseInsensitive); +} + +REGISTER_FUNCTION(GenerateSnowflakeID) +{ + registerSnowflakeIDGenerator(factory); + registerSnowflakeIDGenerator(factory); +} + +} diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 411a3a076ac..f2a82431c0a 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -76,7 +76,7 @@ void setVariant(UUID & uuid) struct FillAllRandomPolicy { static constexpr auto name = "generateUUIDv7NonMonotonic"; - static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), and a random field (74 bit, including a 2-bit variant field "2") to distinguish UUIDs within a millisecond. This function is the fastest generateUUIDv7* function but it gives no monotonicity guarantees within a timestamp.)"; + static constexpr auto description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), and a random field (74 bit, including a 2-bit variant field "2") to distinguish UUIDs within a millisecond. This function is the fastest generateUUIDv7* function but it gives no monotonicity guarantees within a timestamp.)"; struct Data { void generate(UUID & uuid, uint64_t ts) @@ -136,7 +136,7 @@ struct CounterFields struct GlobalCounterPolicy { static constexpr auto name = "generateUUIDv7"; - static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. Function generateUUIDv7 guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; + static constexpr auto description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. Function generateUUIDv7 guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; /// Guarantee counter monotonicity within one timestamp across all threads generating UUIDv7 simultaneously. struct Data @@ -159,7 +159,7 @@ struct GlobalCounterPolicy struct ThreadLocalCounterPolicy { static constexpr auto name = "generateUUIDv7ThreadMonotonic"; - static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. This function behaves like generateUUIDv7 but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate UUIDs.)"; + static constexpr auto description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. This function behaves like generateUUIDv7 but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate UUIDs.)"; /// Guarantee counter monotonicity within one timestamp within the same thread. Faster than GlobalCounterPolicy if a query uses multiple threads. struct Data @@ -186,7 +186,6 @@ class FunctionGenerateUUIDv7Base : public IFunction, public FillPolicy { public: String getName() const final { return FillPolicy::name; } - size_t getNumberOfArguments() const final { return 0; } bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const final { return false; } @@ -198,7 +197,7 @@ public: { FunctionArgumentDescriptors mandatory_args; FunctionArgumentDescriptors optional_args{ - {"expr", nullptr, nullptr, "Arbitrary Expression"} + {"expr", nullptr, nullptr, "Arbitrary expression"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -264,20 +263,20 @@ private: }; template -void registerUUIDv7Generator(auto& factory) +void registerUUIDv7Generator(auto & factory) { static constexpr auto doc_syntax_format = "{}([expression])"; static constexpr auto example_format = "SELECT {}()"; static constexpr auto multiple_example_format = "SELECT {f}(1), {f}(2)"; - FunctionDocumentation::Description doc_description = FillPolicy::doc_description; - FunctionDocumentation::Syntax doc_syntax = fmt::format(doc_syntax_format, FillPolicy::name); - FunctionDocumentation::Arguments doc_arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; - FunctionDocumentation::ReturnedValue doc_returned_value = "A value of type UUID version 7."; - FunctionDocumentation::Examples doc_examples = {{"uuid", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; - FunctionDocumentation::Categories doc_categories = {"UUID"}; + FunctionDocumentation::Description description = FillPolicy::description; + FunctionDocumentation::Syntax syntax = fmt::format(doc_syntax_format, FillPolicy::name); + FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; + FunctionDocumentation::ReturnedValue returned_value = "A value of type UUID version 7."; + FunctionDocumentation::Examples examples = {{"single", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; + FunctionDocumentation::Categories categories = {"UUID"}; - factory.template registerFunction>({doc_description, doc_syntax, doc_arguments, doc_returned_value, doc_examples, doc_categories}, FunctionFactory::CaseInsensitive); + factory.template registerFunction>({description, syntax, arguments, returned_value, examples, categories}, FunctionFactory::CaseInsensitive); } REGISTER_FUNCTION(GenerateUUIDv7) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 6ad54923ab5..9c20ee4cff0 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -4,8 +4,7 @@ #if USE_AZURE_BLOB_STORAGE -#include -#include +#include #include #include #include diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index b8e1134d422..22913125e99 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -52,7 +52,6 @@ std::string toContentEncodingName(CompressionMethod method) case CompressionMethod::None: return ""; } - UNREACHABLE(); } CompressionMethod chooseHTTPCompressionMethod(const std::string & list) diff --git a/src/IO/HadoopSnappyReadBuffer.h b/src/IO/HadoopSnappyReadBuffer.h index 73e52f2c503..bbbb84dd6dd 100644 --- a/src/IO/HadoopSnappyReadBuffer.h +++ b/src/IO/HadoopSnappyReadBuffer.h @@ -88,7 +88,6 @@ public: case Status::TOO_LARGE_COMPRESSED_BLOCK: return "TOO_LARGE_COMPRESSED_BLOCK"; } - UNREACHABLE(); } explicit HadoopSnappyReadBuffer( diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index c52e6bc1441..363f98c46f5 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -29,6 +29,7 @@ struct URI std::string key; std::string version_id; std::string storage_name; + /// Path (or path pattern) in archive if uri is an archive. std::optional archive_pattern; std::string uri_str; diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index eee3da9fb74..78efda4ae57 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -53,7 +53,7 @@ namespace const auto & result = outcome.GetResult(); ObjectInfo object_info; object_info.size = static_cast(result.GetContentLength()); - object_info.last_modification_time = result.GetLastModified().Millis() / 1000; + object_info.last_modification_time = result.GetLastModified().Seconds(); if (with_metadata) object_info.metadata = result.GetMetadata(); diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 4583b2bb0ac..78c51fcb29c 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -174,8 +174,11 @@ void AuthSettings::updateFrom(const AuthSettings & from) if (!from.session_token.empty()) session_token = from.session_token; - headers = from.headers; - region = from.region; + if (!from.headers.empty()) + headers = from.headers; + if (!from.region.empty()) + region = from.region; + server_side_encryption_customer_key_base64 = from.server_side_encryption_customer_key_base64; server_side_encryption_kms_config = from.server_side_encryption_kms_config; diff --git a/src/IO/examples/read_buffer_from_hdfs.cpp b/src/IO/examples/read_buffer_from_hdfs.cpp index c499542fedb..91139ad94eb 100644 --- a/src/IO/examples/read_buffer_from_hdfs.cpp +++ b/src/IO/examples/read_buffer_from_hdfs.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index 447b72ed7c6..4a4d7cc0fc2 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -547,7 +547,7 @@ public: std::unique_ptr getWriteBuffer(String file_name = "file") { S3Settings::RequestSettings request_settings; - request_settings.updateFromSettings(settings); + request_settings.updateFromSettingsIfChanged(settings); client->resetCounters(); diff --git a/src/Interpreters/AggregatedDataVariants.cpp b/src/Interpreters/AggregatedDataVariants.cpp index 87cfdda5948..8f82f15248f 100644 --- a/src/Interpreters/AggregatedDataVariants.cpp +++ b/src/Interpreters/AggregatedDataVariants.cpp @@ -117,8 +117,6 @@ size_t AggregatedDataVariants::size() const APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } - - UNREACHABLE(); } size_t AggregatedDataVariants::sizeWithoutOverflowRow() const @@ -136,8 +134,6 @@ size_t AggregatedDataVariants::sizeWithoutOverflowRow() const APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } - - UNREACHABLE(); } const char * AggregatedDataVariants::getMethodName() const @@ -155,8 +151,6 @@ const char * AggregatedDataVariants::getMethodName() const APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } - - UNREACHABLE(); } bool AggregatedDataVariants::isTwoLevel() const @@ -174,8 +168,6 @@ bool AggregatedDataVariants::isTwoLevel() const APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } - - UNREACHABLE(); } bool AggregatedDataVariants::isConvertibleToTwoLevel() const diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 9459029dc4c..61a356fa3c3 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -799,7 +799,6 @@ String FileSegment::stateToString(FileSegment::State state) case FileSegment::State::DETACHED: return "DETACHED"; } - UNREACHABLE(); } bool FileSegment::assertCorrectness() const diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 4b10bfd3dcd..a3fe8c2e779 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -126,6 +126,11 @@ bool astContainsSystemTables(ASTPtr ast, ContextPtr context) namespace { +bool isQueryCacheRelatedSetting(const String & setting_name) +{ + return setting_name.starts_with("query_cache_") || setting_name.ends_with("_query_cache"); +} + class RemoveQueryCacheSettingsMatcher { public: @@ -141,7 +146,7 @@ public: auto is_query_cache_related_setting = [](const auto & change) { - return change.name.starts_with("query_cache_") || change.name.ends_with("_query_cache"); + return isQueryCacheRelatedSetting(change.name); }; std::erase_if(set_clause->changes, is_query_cache_related_setting); @@ -177,11 +182,11 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } -IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database) +IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database, const Settings & settings) { ast = removeQueryCacheSettings(ast); - /// Hash the AST, it must consider aliases (issue #56258) + /// Hash the AST, we must consider aliases (issue #56258) SipHash hash; ast->updateTreeHash(hash, /*ignore_aliases=*/ false); @@ -189,6 +194,25 @@ IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database) /// tables (issue #64136) hash.update(current_database); + /// Finally, hash the (changed) settings as they might affect the query result (e.g. think of settings `additional_table_filters` and `limit`). + /// Note: allChanged() returns the settings in random order. Also, update()-s of the composite hash must be done in deterministic order. + /// Therefore, collect and sort the settings first, then hash them. + Settings::Range changed_settings = settings.allChanged(); + std::vector> changed_settings_sorted; /// (name, value) + for (const auto & setting : changed_settings) + { + const String & name = setting.getName(); + const String & value = setting.getValueString(); + if (!isQueryCacheRelatedSetting(name)) /// see removeQueryCacheSettings() why this is a good idea + changed_settings_sorted.push_back({name, value}); + } + std::sort(changed_settings_sorted.begin(), changed_settings_sorted.end(), [](auto & lhs, auto & rhs) { return lhs.first < rhs.first; }); + for (const auto & setting : changed_settings_sorted) + { + hash.update(setting.first); + hash.update(setting.second); + } + return getSipHash128AsPair(hash); } @@ -204,12 +228,13 @@ String queryStringFromAST(ASTPtr ast) QueryCache::Key::Key( ASTPtr ast_, const String & current_database, + const Settings & settings, Block header_, std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, bool is_compressed_) - : ast_hash(calculateAstHash(ast_, current_database)) + : ast_hash(calculateAstHash(ast_, current_database, settings)) , header(header_) , user_id(user_id_) , current_user_roles(current_user_roles_) @@ -220,8 +245,8 @@ QueryCache::Key::Key( { } -QueryCache::Key::Key(ASTPtr ast_, const String & current_database, std::optional user_id_, const std::vector & current_user_roles_) - : QueryCache::Key(ast_, current_database, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles +QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_) + : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index b5b6f477137..461197cac32 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -14,6 +14,8 @@ namespace DB { +struct Settings; + /// Does AST contain non-deterministic functions like rand() and now()? bool astContainsNonDeterministicFunctions(ASTPtr ast, ContextPtr context); @@ -89,6 +91,7 @@ public: /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, const String & current_database, + const Settings & settings, Block header_, std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, @@ -96,7 +99,7 @@ public: bool is_compressed); /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). - Key(ASTPtr ast_, const String & current_database, std::optional user_id_, const std::vector & current_user_roles_); + Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_); bool operator==(const Key & other) const; }; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 4bbda982f5b..13e6fa87051 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -403,6 +403,10 @@ void executeQueryWithParallelReplicas( ContextPtr context, std::shared_ptr storage_limits) { + auto logger = getLogger("executeQueryWithParallelReplicas"); + LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas", + storage_id.getNameForLogs(), header.dumpStructure(), query_ast->formatForLogging(), processed_stage); + const auto & settings = context->getSettingsRef(); /// check cluster for parallel replicas diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 4eacbae7a30..d53ff4b0227 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -309,7 +309,6 @@ ComparisonGraphCompareResult ComparisonGraph::pathToCompareResult(Path pat case Path::GREATER: return inverse ? ComparisonGraphCompareResult::LESS : ComparisonGraphCompareResult::GREATER; case Path::GREATER_OR_EQUAL: return inverse ? ComparisonGraphCompareResult::LESS_OR_EQUAL : ComparisonGraphCompareResult::GREATER_OR_EQUAL; } - UNREACHABLE(); } template diff --git a/src/Interpreters/FilesystemCacheLog.cpp b/src/Interpreters/FilesystemCacheLog.cpp index 80fe1c3a8ef..aa489351a98 100644 --- a/src/Interpreters/FilesystemCacheLog.cpp +++ b/src/Interpreters/FilesystemCacheLog.cpp @@ -26,7 +26,6 @@ static String typeToString(FilesystemCacheLogElement::CacheType type) case FilesystemCacheLogElement::CacheType::WRITE_THROUGH_CACHE: return "WRITE_THROUGH_CACHE"; } - UNREACHABLE(); } ColumnsDescription FilesystemCacheLogElement::getColumnsDescription() diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 3a21c13db5e..75da8bbc3e7 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -705,7 +705,6 @@ namespace APPLY_FOR_JOIN_VARIANTS(M) #undef M } - UNREACHABLE(); } } @@ -2641,8 +2640,6 @@ private: default: throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type); } - - UNREACHABLE(); } template diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 86db8943926..a0996556f9a 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -322,8 +322,6 @@ public: APPLY_FOR_JOIN_VARIANTS(M) #undef M } - - UNREACHABLE(); } size_t getTotalByteCountImpl(Type which) const @@ -338,8 +336,6 @@ public: APPLY_FOR_JOIN_VARIANTS(M) #undef M } - - UNREACHABLE(); } size_t getBufferSizeInCells(Type which) const @@ -354,8 +350,6 @@ public: APPLY_FOR_JOIN_VARIANTS(M) #undef M } - - UNREACHABLE(); } /// NOLINTEND(bugprone-macro-parentheses) }; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 61ca606186a..b30fc8bc092 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1500,7 +1500,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, validateVirtualColumns(*res); - if (!res->supportsDynamicSubcolumnsDeprecated() && hasDynamicSubcolumns(res->getInMemoryMetadataPtr()->getColumns())) + if (!res->supportsDynamicSubcolumnsDeprecated() && hasDynamicSubcolumns(res->getInMemoryMetadataPtr()->getColumns()) && mode <= LoadingStrictnessLevel::CREATE) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column of type Object, " diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index ee774994145..9cfb8e486cb 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes extern const int TABLE_IS_READ_ONLY; extern const int SUPPORT_IS_DISABLED; extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; } @@ -107,7 +108,19 @@ BlockIO InterpreterDeleteQuery::execute() } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "DELETE query is not supported for table {}", table->getStorageID().getFullTableName()); + /// Currently just better exception for the case of a table with projection, + /// can act differently according to the setting. + if (table->hasProjection()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DELETE query is not supported for table {} as it has projections. " + "User should drop all the projections manually before running the query", + table->getStorageID().getFullTableName()); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "DELETE query is not supported for table {}", + table->getStorageID().getFullTableName()); } } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 498030a1552..d3526941b33 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -51,11 +51,12 @@ #include #include #include -#include #include -#include +#include +#include +#include +#include #include -#include #include #include #include @@ -500,17 +501,17 @@ BlockIO InterpreterSystemQuery::execute() StorageFile::getSchemaCache(getContext()).clear(); #if USE_AWS_S3 if (caches_to_drop.contains("S3")) - StorageS3::getSchemaCache(getContext()).clear(); + StorageObjectStorage::getSchemaCache(getContext(), StorageS3Configuration::type_name).clear(); #endif #if USE_HDFS if (caches_to_drop.contains("HDFS")) - StorageHDFS::getSchemaCache(getContext()).clear(); + StorageObjectStorage::getSchemaCache(getContext(), StorageHDFSConfiguration::type_name).clear(); #endif if (caches_to_drop.contains("URL")) StorageURL::getSchemaCache(getContext()).clear(); #if USE_AZURE_BLOB_STORAGE if (caches_to_drop.contains("AZURE")) - StorageAzureBlob::getSchemaCache(getContext()).clear(); + StorageObjectStorage::getSchemaCache(getContext(), StorageAzureConfiguration::type_name).clear(); #endif break; } diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index d31ace758c4..13872fbe3f5 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -33,7 +33,6 @@ BlockIO InterpreterTransactionControlQuery::execute() case ASTTransactionControl::SET_SNAPSHOT: return executeSetSnapshot(session_context, tcl.snapshot); } - UNREACHABLE(); } BlockIO InterpreterTransactionControlQuery::executeBegin(ContextMutablePtr session_context) diff --git a/src/Interpreters/SetVariants.cpp b/src/Interpreters/SetVariants.cpp index 64796a013f1..c600d096160 100644 --- a/src/Interpreters/SetVariants.cpp +++ b/src/Interpreters/SetVariants.cpp @@ -41,8 +41,6 @@ size_t SetVariantsTemplate::getTotalRowCount() const APPLY_FOR_SET_VARIANTS(M) #undef M } - - UNREACHABLE(); } template @@ -57,8 +55,6 @@ size_t SetVariantsTemplate::getTotalByteCount() const APPLY_FOR_SET_VARIANTS(M) #undef M } - - UNREACHABLE(); } template diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 56f08dbb902..59d012a0a0e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1093,6 +1093,15 @@ static std::tuple executeQueryImpl( && (ast->as() || ast->as()); QueryCache::Usage query_cache_usage = QueryCache::Usage::None; + /// If the query runs with "use_query_cache = 1", we first probe if the query cache already contains the query result (if yes: + /// return result from cache). If doesn't, we execute the query normally and write the result into the query cache. Both steps use a + /// hash of the AST, the current database and the settings as cache key. Unfortunately, the settings are in some places internally + /// modified between steps 1 and 2 (= during query execution) - this is silly but hard to forbid. As a result, the hashes no longer + /// match and the cache is rendered ineffective. Therefore make a copy of the settings and use it for steps 1 and 2. + std::optional settings_copy; + if (can_use_query_cache) + settings_copy = settings; + if (!async_insert) { /// If it is a non-internal SELECT, and passive (read) use of the query cache is enabled, and the cache knows the query, then set @@ -1101,7 +1110,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getCurrentDatabase(), context->getUserID(), context->getCurrentRoles()); + QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1224,7 +1233,7 @@ static std::tuple executeQueryImpl( && (!ast_contains_system_tables || system_table_handling == QueryCacheSystemTableHandling::Save)) { QueryCache::Key key( - ast, context->getCurrentDatabase(), res.pipeline.getHeader(), + ast, context->getCurrentDatabase(), *settings_copy, res.pipeline.getHeader(), context->getUserID(), context->getCurrentRoles(), settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index 701bde8cebd..eb095b5dbbc 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -40,8 +40,6 @@ public: case TableOverride: return "EXPLAIN TABLE OVERRIDE"; case CurrentTransaction: return "EXPLAIN CURRENT TRANSACTION"; } - - UNREACHABLE(); } static ExplainKind fromString(const String & str) diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index 34855a7ce20..5f2bd50524c 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -42,7 +42,7 @@ Token quotedString(const char *& pos, const char * const token_begin, const char continue; } - UNREACHABLE(); + chassert(false); } } @@ -538,8 +538,6 @@ const char * getTokenName(TokenType type) APPLY_FOR_TOKENS(M) #undef M } - - UNREACHABLE(); } diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 0b5767407e7..2a95234057c 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -444,7 +444,6 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, for (auto & interpolate_node : interpolate_list_node.getNodes()) { auto & interpolate_node_typed = interpolate_node->as(); - interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); } diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 98cbdeaaa4b..6b7f1f5206c 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -657,7 +657,6 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Msgpack extension type {:x} is not supported", object_ext.type()); } } - UNREACHABLE(); } std::optional MsgPackSchemaReader::readRowAndGetDataTypes() diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index 8b160153733..5ab5e5277aa 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -36,8 +36,6 @@ std::string IProcessor::statusToName(Status status) case Status::ExpandPipeline: return "ExpandPipeline"; } - - UNREACHABLE(); } } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6f0fa55c349..24ea8c25fb6 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1136,8 +1136,6 @@ static void addMergingFinal( return std::make_shared(header, num_outputs, sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, merging_params.graphite_params, now); } - - UNREACHABLE(); }; pipe.addTransform(get_merging_processor()); @@ -2125,8 +2123,6 @@ static const char * indexTypeToString(ReadFromMergeTree::IndexType type) case ReadFromMergeTree::IndexType::Skip: return "Skip"; } - - UNREACHABLE(); } static const char * readTypeToString(ReadFromMergeTree::ReadType type) @@ -2142,8 +2138,6 @@ static const char * readTypeToString(ReadFromMergeTree::ReadType type) case ReadFromMergeTree::ReadType::ParallelReplicas: return "Parallel"; } - - UNREACHABLE(); } void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index b4e35af85d6..84c2515e8ca 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -386,6 +386,8 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( chassert(cluster->getShardCount() == 1); std::vector description; + description.push_back(fmt::format("query: {}", formattedAST(query_ast))); + for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools) description.push_back(fmt::format("Replica: {}", pool->getHost())); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index d1bd70fd0b2..ac5e144bf4a 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -86,8 +86,6 @@ static String totalsModeToString(TotalsMode totals_mode, double auto_include_thr case TotalsMode::AFTER_HAVING_AUTO: return "after_having_auto threshold " + std::to_string(auto_include_threshold); } - - UNREACHABLE(); } void TotalsHavingStep::describeActions(FormatSettings & settings) const diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 05fd2a7254f..bb38c3e1dc5 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -67,7 +67,6 @@ static FillColumnDescription::StepFunction getStepFunction( FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE } - UNREACHABLE(); } static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & type) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index cdcfad4442c..a1a886fb4f7 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -898,8 +898,6 @@ static std::exception_ptr addStorageToException(std::exception_ptr ptr, const St { return std::current_exception(); } - - UNREACHABLE(); } void FinalizingViewsTransform::work() diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 3db935729b4..e3a820340ad 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.h b/src/Storages/DataLakes/DeltaLakeMetadataParser.h deleted file mode 100644 index df7276b90b4..00000000000 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.h +++ /dev/null @@ -1,22 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -template -struct DeltaLakeMetadataParser -{ -public: - DeltaLakeMetadataParser(); - - Strings getFiles(const Configuration & configuration, ContextPtr context); - -private: - struct Impl; - std::shared_ptr impl; -}; - -} diff --git a/src/Storages/DataLakes/HudiMetadataParser.cpp b/src/Storages/DataLakes/HudiMetadataParser.cpp deleted file mode 100644 index 699dfe8fda0..00000000000 --- a/src/Storages/DataLakes/HudiMetadataParser.cpp +++ /dev/null @@ -1,116 +0,0 @@ -#include -#include -#include -#include -#include -#include "config.h" -#include -#include - -#if USE_AWS_S3 -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -template -struct HudiMetadataParser::Impl -{ - /** - * Useful links: - * - https://hudi.apache.org/tech-specs/ - * - https://hudi.apache.org/docs/file_layouts/ - */ - - /** - * Hudi tables store metadata files and data files. - * Metadata files are stored in .hoodie/metadata directory. Though unlike DeltaLake and Iceberg, - * metadata is not required in order to understand which files we need to read, moreover, - * for Hudi metadata does not always exist. - * - * There can be two types of data files - * 1. base files (columnar file formats like Apache Parquet/Orc) - * 2. log files - * Currently we support reading only `base files`. - * Data file name format: - * [File Id]_[File Write Token]_[Transaction timestamp].[File Extension] - * - * To find needed parts we need to find out latest part file for every file group for every partition. - * Explanation why: - * Hudi reads in and overwrites the entire table/partition with each update. - * Hudi controls the number of file groups under a single partition according to the - * hoodie.parquet.max.file.size option. Once a single Parquet file is too large, Hudi creates a second file group. - * Each file group is identified by File Id. - */ - Strings processMetadataFiles(const Configuration & configuration) - { - auto log = getLogger("HudiMetadataParser"); - - const auto keys = MetadataReadHelper::listFiles(configuration, "", Poco::toLower(configuration.format)); - - using Partition = std::string; - using FileID = std::string; - struct FileInfo - { - String key; - UInt64 timestamp = 0; - }; - std::unordered_map> data_files; - - for (const auto & key : keys) - { - auto key_file = std::filesystem::path(key); - Strings file_parts; - const String stem = key_file.stem(); - splitInto<'_'>(file_parts, stem); - if (file_parts.size() != 3) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format for file: {}", key); - - const auto partition = key_file.parent_path().stem(); - const auto & file_id = file_parts[0]; - const auto timestamp = parse(file_parts[2]); - - auto & file_info = data_files[partition][file_id]; - if (file_info.timestamp == 0 || file_info.timestamp < timestamp) - { - file_info.key = std::move(key); - file_info.timestamp = timestamp; - } - } - - Strings result; - for (auto & [partition, partition_data] : data_files) - { - LOG_TRACE(log, "Adding {} data files from partition {}", partition, partition_data.size()); - for (auto & [file_id, file_data] : partition_data) - result.push_back(std::move(file_data.key)); - } - return result; - } -}; - - -template -HudiMetadataParser::HudiMetadataParser() : impl(std::make_unique()) -{ -} - -template -Strings HudiMetadataParser::getFiles(const Configuration & configuration, ContextPtr) -{ - return impl->processMetadataFiles(configuration); -} - -template HudiMetadataParser::HudiMetadataParser(); -template Strings HudiMetadataParser::getFiles( - const StorageS3::Configuration & configuration, ContextPtr); - -} - -#endif diff --git a/src/Storages/DataLakes/HudiMetadataParser.h b/src/Storages/DataLakes/HudiMetadataParser.h deleted file mode 100644 index 6727ba2f718..00000000000 --- a/src/Storages/DataLakes/HudiMetadataParser.h +++ /dev/null @@ -1,22 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -template -struct HudiMetadataParser -{ -public: - HudiMetadataParser(); - - Strings getFiles(const Configuration & configuration, ContextPtr context); - -private: - struct Impl; - std::shared_ptr impl; -}; - -} diff --git a/src/Storages/DataLakes/IStorageDataLake.h b/src/Storages/DataLakes/IStorageDataLake.h deleted file mode 100644 index 2147f2c9e6b..00000000000 --- a/src/Storages/DataLakes/IStorageDataLake.h +++ /dev/null @@ -1,136 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -template -class IStorageDataLake : public Storage -{ -public: - static constexpr auto name = Name::name; - using Configuration = typename Storage::Configuration; - - template - explicit IStorageDataLake(const Configuration & configuration_, ContextPtr context_, LoadingStrictnessLevel mode, Args && ...args) - : Storage(getConfigurationForDataRead(configuration_, context_, {}, mode), context_, std::forward(args)...) - , base_configuration(configuration_) - , log(getLogger(getName())) {} // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall) - - template - static StoragePtr create(const Configuration & configuration_, ContextPtr context_, LoadingStrictnessLevel mode, Args && ...args) - { - return std::make_shared>(configuration_, context_, mode, std::forward(args)...); - } - - String getName() const override { return name; } - - static ColumnsDescription getTableStructureFromData( - Configuration & base_configuration, - const std::optional & format_settings, - const ContextPtr & local_context) - { - auto configuration = getConfigurationForDataRead(base_configuration, local_context); - return Storage::getTableStructureFromData(configuration, format_settings, local_context); - } - - static Configuration getConfiguration(ASTs & engine_args, const ContextPtr & local_context) - { - return Storage::getConfiguration(engine_args, local_context, /* get_format_from_file */false); - } - - Configuration updateConfigurationAndGetCopy(const ContextPtr & local_context) override - { - std::lock_guard lock(configuration_update_mutex); - updateConfigurationImpl(local_context); - return Storage::getConfigurationCopy(); - } - - void updateConfiguration(const ContextPtr & local_context) override - { - std::lock_guard lock(configuration_update_mutex); - updateConfigurationImpl(local_context); - } - -private: - static Configuration getConfigurationForDataRead( - const Configuration & base_configuration, const ContextPtr & local_context, const Strings & keys = {}, - LoadingStrictnessLevel mode = LoadingStrictnessLevel::CREATE) - { - auto configuration{base_configuration}; - configuration.update(local_context); - configuration.static_configuration = true; - - try - { - if (keys.empty()) - configuration.keys = getDataFiles(configuration, local_context); - else - configuration.keys = keys; - - LOG_TRACE( - getLogger("DataLake"), - "New configuration path: {}, keys: {}", - configuration.getPath(), fmt::join(configuration.keys, ", ")); - - configuration.connect(local_context); - return configuration; - } - catch (...) - { - if (mode <= LoadingStrictnessLevel::CREATE) - throw; - tryLogCurrentException(__PRETTY_FUNCTION__); - return configuration; - } - } - - static Strings getDataFiles(const Configuration & configuration, const ContextPtr & local_context) - { - return MetadataParser().getFiles(configuration, local_context); - } - - void updateConfigurationImpl(const ContextPtr & local_context) - { - const bool updated = base_configuration.update(local_context); - auto new_keys = getDataFiles(base_configuration, local_context); - - if (!updated && new_keys == Storage::getConfigurationCopy().keys) - return; - - Storage::useConfiguration(getConfigurationForDataRead(base_configuration, local_context, new_keys)); - } - - Configuration base_configuration; - std::mutex configuration_update_mutex; - LoggerPtr log; -}; - - -template -static StoragePtr createDataLakeStorage(const StorageFactory::Arguments & args) -{ - auto configuration = DataLake::getConfiguration(args.engine_args, args.getLocalContext()); - - /// Data lakes use parquet format, no need for schema inference. - if (configuration.format == "auto") - configuration.format = "Parquet"; - - return DataLake::create(configuration, args.getContext(), args.mode, args.table_id, args.columns, args.constraints, - args.comment, getFormatSettings(args.getContext())); -} - -} - -#endif diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp deleted file mode 100644 index 19cd97c3d4f..00000000000 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp +++ /dev/null @@ -1,90 +0,0 @@ -#include - -#if USE_AWS_S3 && USE_AVRO - -namespace DB -{ - -StoragePtr StorageIceberg::create( - const DB::StorageIceberg::Configuration & base_configuration, - DB::ContextPtr context_, - LoadingStrictnessLevel mode, - const DB::StorageID & table_id_, - const DB::ColumnsDescription & columns_, - const DB::ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_) -{ - auto configuration{base_configuration}; - configuration.update(context_); - std::unique_ptr metadata; - NamesAndTypesList schema_from_metadata; - try - { - metadata = parseIcebergMetadata(configuration, context_); - schema_from_metadata = metadata->getTableSchema(); - configuration.keys = metadata->getDataFiles(); - } - catch (...) - { - if (mode <= LoadingStrictnessLevel::CREATE) - throw; - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - return std::make_shared( - std::move(metadata), - configuration, - context_, - table_id_, - columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, - constraints_, - comment, - format_settings_); -} - -StorageIceberg::StorageIceberg( - std::unique_ptr metadata_, - const Configuration & configuration_, - ContextPtr context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_) - : StorageS3(configuration_, context_, table_id_, columns_, constraints_, comment, format_settings_) - , current_metadata(std::move(metadata_)) - , base_configuration(configuration_) -{ -} - -ColumnsDescription StorageIceberg::getTableStructureFromData( - Configuration & base_configuration, - const std::optional &, - const ContextPtr & local_context) -{ - auto configuration{base_configuration}; - configuration.update(local_context); - auto metadata = parseIcebergMetadata(configuration, local_context); - return ColumnsDescription(metadata->getTableSchema()); -} - -void StorageIceberg::updateConfigurationImpl(const ContextPtr & local_context) -{ - const bool updated = base_configuration.update(local_context); - auto new_metadata = parseIcebergMetadata(base_configuration, local_context); - - if (!current_metadata || new_metadata->getVersion() != current_metadata->getVersion()) - current_metadata = std::move(new_metadata); - else if (!updated) - return; - - auto updated_configuration{base_configuration}; - /// If metadata wasn't changed, we won't list data files again. - updated_configuration.keys = current_metadata->getDataFiles(); - StorageS3::useConfiguration(updated_configuration); -} - -} - -#endif diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.h b/src/Storages/DataLakes/Iceberg/StorageIceberg.h deleted file mode 100644 index 9e3885124d6..00000000000 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.h +++ /dev/null @@ -1,85 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 && USE_AVRO - -# include -# include -# include -# include -# include -# include -# include - - -namespace DB -{ - -/// Storage for read-only integration with Apache Iceberg tables in Amazon S3 (see https://iceberg.apache.org/) -/// Right now it's implemented on top of StorageS3 and right now it doesn't support -/// many Iceberg features like schema evolution, partitioning, positional and equality deletes. -/// TODO: Implement Iceberg as a separate storage using IObjectStorage -/// (to support all object storages, not only S3) and add support for missing Iceberg features. -class StorageIceberg : public StorageS3 -{ -public: - static constexpr auto name = "Iceberg"; - - using Configuration = StorageS3::Configuration; - - static StoragePtr create(const Configuration & base_configuration, - ContextPtr context_, - LoadingStrictnessLevel mode, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_); - - StorageIceberg( - std::unique_ptr metadata_, - const Configuration & configuration_, - ContextPtr context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_); - - String getName() const override { return name; } - - static ColumnsDescription getTableStructureFromData( - Configuration & base_configuration, - const std::optional &, - const ContextPtr & local_context); - - static Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context) - { - return StorageS3::getConfiguration(engine_args, local_context, /* get_format_from_file */false); - } - - Configuration updateConfigurationAndGetCopy(const ContextPtr & local_context) override - { - std::lock_guard lock(configuration_update_mutex); - updateConfigurationImpl(local_context); - return StorageS3::getConfigurationCopy(); - } - - void updateConfiguration(const ContextPtr & local_context) override - { - std::lock_guard lock(configuration_update_mutex); - updateConfigurationImpl(local_context); - } - -private: - void updateConfigurationImpl(const ContextPtr & local_context); - - std::unique_ptr current_metadata; - Configuration base_configuration; - std::mutex configuration_update_mutex; -}; - -} - -#endif diff --git a/src/Storages/DataLakes/S3MetadataReader.cpp b/src/Storages/DataLakes/S3MetadataReader.cpp deleted file mode 100644 index 62a486951fe..00000000000 --- a/src/Storages/DataLakes/S3MetadataReader.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int S3_ERROR; -} - -std::shared_ptr -S3DataLakeMetadataReadHelper::createReadBuffer(const String & key, ContextPtr context, const StorageS3::Configuration & base_configuration) -{ - S3Settings::RequestSettings request_settings; - request_settings.max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; - return std::make_shared( - base_configuration.client, - base_configuration.url.bucket, - key, - base_configuration.url.version_id, - request_settings, - context->getReadSettings()); -} - -bool S3DataLakeMetadataReadHelper::exists(const String & key, const StorageS3::Configuration & configuration) -{ - return S3::objectExists(*configuration.client, configuration.url.bucket, key); -} - -std::vector S3DataLakeMetadataReadHelper::listFiles( - const StorageS3::Configuration & base_configuration, const String & prefix, const String & suffix) -{ - const auto & table_path = base_configuration.url.key; - const auto & bucket = base_configuration.url.bucket; - const auto & client = base_configuration.client; - - std::vector res; - S3::ListObjectsV2Request request; - Aws::S3::Model::ListObjectsV2Outcome outcome; - - request.SetBucket(bucket); - request.SetPrefix(std::filesystem::path(table_path) / prefix); - - bool is_finished{false}; - while (!is_finished) - { - outcome = client->ListObjectsV2(request); - if (!outcome.IsSuccess()) - throw S3Exception( - outcome.GetError().GetErrorType(), - "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", - quoteString(bucket), - quoteString(base_configuration.url.key), - backQuote(outcome.GetError().GetExceptionName()), - quoteString(outcome.GetError().GetMessage())); - - const auto & result_batch = outcome.GetResult().GetContents(); - for (const auto & obj : result_batch) - { - const auto & filename = obj.GetKey(); - if (filename.ends_with(suffix)) - res.push_back(filename); - } - - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - is_finished = !outcome.GetResult().GetIsTruncated(); - } - - LOG_TRACE(getLogger("S3DataLakeMetadataReadHelper"), "Listed {} files", res.size()); - - return res; -} - -} -#endif diff --git a/src/Storages/DataLakes/S3MetadataReader.h b/src/Storages/DataLakes/S3MetadataReader.h deleted file mode 100644 index c29a66b3813..00000000000 --- a/src/Storages/DataLakes/S3MetadataReader.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include - -#if USE_AWS_S3 - -#include - -namespace DB -{ - -class ReadBuffer; - -struct S3DataLakeMetadataReadHelper -{ - static std::shared_ptr createReadBuffer( - const String & key, ContextPtr context, const StorageS3::Configuration & base_configuration); - - static bool exists(const String & key, const StorageS3::Configuration & configuration); - - static std::vector listFiles(const StorageS3::Configuration & configuration, const std::string & prefix = "", const std::string & suffix = ""); -}; -} - -#endif diff --git a/src/Storages/DataLakes/StorageDeltaLake.h b/src/Storages/DataLakes/StorageDeltaLake.h deleted file mode 100644 index 8b4ba28d6f7..00000000000 --- a/src/Storages/DataLakes/StorageDeltaLake.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include -#include -#include "config.h" - -#if USE_AWS_S3 -#include -#include -#endif - -namespace DB -{ - -struct StorageDeltaLakeName -{ - static constexpr auto name = "DeltaLake"; -}; - -#if USE_AWS_S3 && USE_PARQUET -using StorageDeltaLakeS3 = IStorageDataLake>; -#endif - -} diff --git a/src/Storages/DataLakes/StorageHudi.h b/src/Storages/DataLakes/StorageHudi.h deleted file mode 100644 index 84666f51405..00000000000 --- a/src/Storages/DataLakes/StorageHudi.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include -#include -#include "config.h" - -#if USE_AWS_S3 -#include -#include -#endif - -namespace DB -{ - -struct StorageHudiName -{ - static constexpr auto name = "Hudi"; -}; - -#if USE_AWS_S3 -using StorageHudiS3 = IStorageDataLake>; -#endif - -} diff --git a/src/Storages/DataLakes/registerDataLakes.cpp b/src/Storages/DataLakes/registerDataLakes.cpp deleted file mode 100644 index 118600f7212..00000000000 --- a/src/Storages/DataLakes/registerDataLakes.cpp +++ /dev/null @@ -1,50 +0,0 @@ -#include -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include - - -namespace DB -{ - -#define REGISTER_DATA_LAKE_STORAGE(STORAGE, NAME) \ - factory.registerStorage( \ - NAME, \ - [](const StorageFactory::Arguments & args) \ - { \ - return createDataLakeStorage(args);\ - }, \ - { \ - .supports_settings = false, \ - .supports_schema_inference = true, \ - .source_access_type = AccessType::S3, \ - }); - -#if USE_PARQUET -void registerStorageDeltaLake(StorageFactory & factory) -{ - REGISTER_DATA_LAKE_STORAGE(StorageDeltaLakeS3, StorageDeltaLakeName::name) -} -#endif - -#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. - -void registerStorageIceberg(StorageFactory & factory) -{ - REGISTER_DATA_LAKE_STORAGE(StorageIceberg, StorageIceberg::name) -} - -#endif - -void registerStorageHudi(StorageFactory & factory) -{ - REGISTER_DATA_LAKE_STORAGE(StorageHudiS3, StorageHudiName::name) -} - -} - -#endif diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp deleted file mode 100644 index 33bde34b4f9..00000000000 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ /dev/null @@ -1,1208 +0,0 @@ -#include "config.h" - -#if USE_HDFS - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include - -#include - -namespace fs = std::filesystem; - -namespace ProfileEvents -{ - extern const Event EngineFileLikeReadFiles; -} - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ACCESS_DENIED; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; - extern const int CANNOT_COMPILE_REGEXP; - extern const int CANNOT_DETECT_FORMAT; -} -namespace -{ - struct HDFSFileInfoDeleter - { - /// Can have only one entry (see hdfsGetPathInfo()) - void operator()(hdfsFileInfo * info) { hdfsFreeFileInfo(info, 1); } - }; - using HDFSFileInfoPtr = std::unique_ptr; - - /* Recursive directory listing with matched paths as a result. - * Have the same method in StorageFile. - */ - std::vector LSWithRegexpMatching( - const String & path_for_ls, - const HDFSFSPtr & fs, - const String & for_match) - { - std::vector result; - - const size_t first_glob_pos = for_match.find_first_of("*?{"); - - if (first_glob_pos == std::string::npos) - { - const String path = fs::path(path_for_ls + for_match.substr(1)).lexically_normal(); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path.c_str())); - if (hdfs_info) // NOLINT - { - result.push_back(StorageHDFS::PathWithInfo{ - String(path), - StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}}); - } - return result; - } - - const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); - const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' - - const size_t next_slash_after_glob_pos = suffix_with_globs.find('/', 1); - - const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); - - re2::RE2 matcher(makeRegexpPatternFromGlobs(current_glob)); - if (!matcher.ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); - - HDFSFileInfo ls; - ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); - if (ls.file_info == nullptr && errno != ENOENT) // NOLINT - { - // ignore file not found exception, keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. - throw Exception( - ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", prefix_without_globs, String(hdfsGetLastError())); - } - - if (!ls.file_info && ls.length > 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); - for (int i = 0; i < ls.length; ++i) - { - const String full_path = fs::path(ls.file_info[i].mName).lexically_normal(); - const size_t last_slash = full_path.rfind('/'); - const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; - const bool is_directory = ls.file_info[i].mKind == 'D'; - /// Condition with type of current file_info means what kind of path is it in current iteration of ls - if (!is_directory && !looking_for_directory) - { - if (re2::RE2::FullMatch(file_name, matcher)) - result.push_back(StorageHDFS::PathWithInfo{ - String(full_path), - StorageHDFS::PathInfo{ls.file_info[i].mLastMod, static_cast(ls.file_info[i].mSize)}}); - } - else if (is_directory && looking_for_directory) - { - if (re2::RE2::FullMatch(file_name, matcher)) - { - std::vector result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, - suffix_with_globs.substr(next_slash_after_glob_pos)); - /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); - } - } - } - - return result; - } - - std::pair getPathFromUriAndUriWithoutPath(const String & uri) - { - auto pos = uri.find("//"); - if (pos != std::string::npos && pos + 2 < uri.length()) - { - pos = uri.find('/', pos + 2); - if (pos != std::string::npos) - return {uri.substr(pos), uri.substr(0, pos)}; - } - - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage HDFS requires valid URL to be set"); - } - - std::vector getPathsList(const String & path_from_uri, const String & uri_without_path, ContextPtr context) - { - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - Strings paths = expandSelectionGlob(path_from_uri); - - std::vector res; - - for (const auto & path : paths) - { - auto part_of_res = LSWithRegexpMatching("/", fs, path); - res.insert(res.end(), part_of_res.begin(), part_of_res.end()); - } - return res; - } -} - -StorageHDFS::StorageHDFS( - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - const ContextPtr & context_, - const String & compression_method_, - const bool distributed_processing_, - ASTPtr partition_by_) - : IStorage(table_id_) - , WithContext(context_) - , uris({uri_}) - , format_name(format_name_) - , compression_method(compression_method_) - , distributed_processing(distributed_processing_) - , partition_by(partition_by_) -{ - if (format_name != "auto") - FormatFactory::instance().checkFormatName(format_name); - context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); - checkHDFSURL(uri_); - - String path = uri_.substr(uri_.find('/', uri_.find("//") + 2)); - is_path_with_globs = path.find_first_of("*?{") != std::string::npos; - - StorageInMemoryMetadata storage_metadata; - - if (columns_.empty()) - { - ColumnsDescription columns; - if (format_name == "auto") - std::tie(columns, format_name) = getTableStructureAndFormatFromData(uri_, compression_method_, context_); - else - columns = getTableStructureFromData(format_name, uri_, compression_method, context_); - - storage_metadata.setColumns(columns); - } - else - { - if (format_name == "auto") - format_name = getTableStructureAndFormatFromData(uri_, compression_method_, context_).second; - - /// We don't allow special columns in HDFS storage. - if (!columns_.hasOnlyOrdinary()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine HDFS doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL"); - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); -} - -namespace -{ - class ReadBufferIterator : public IReadBufferIterator, WithContext - { - public: - ReadBufferIterator( - const std::vector & paths_with_info_, - const String & uri_without_path_, - std::optional format_, - const String & compression_method_, - const ContextPtr & context_) - : WithContext(context_) - , paths_with_info(paths_with_info_) - , uri_without_path(uri_without_path_) - , format(std::move(format_)) - , compression_method(compression_method_) - { - } - - Data next() override - { - bool is_first = current_index == 0; - /// For default mode check cached columns for all paths on first iteration. - if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(paths_with_info)) - return {nullptr, cached_columns, format}; - } - - StorageHDFS::PathWithInfo path_with_info; - - while (true) - { - if (current_index == paths_with_info.size()) - { - if (is_first) - { - if (format) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The table structure cannot be extracted from a {} format file, because all files are empty. " - "You can specify table structure manually", *format); - - throw Exception( - ErrorCodes::CANNOT_DETECT_FORMAT, - "The data format cannot be detected by the contents of the files, because all files are empty. You can specify table structure manually"); - } - return {nullptr, std::nullopt, format}; - } - - path_with_info = paths_with_info[current_index++]; - if (getContext()->getSettingsRef().hdfs_skip_empty_files && path_with_info.info && path_with_info.info->size == 0) - continue; - - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - std::vector paths = {path_with_info}; - if (auto cached_columns = tryGetColumnsFromCache(paths)) - return {nullptr, cached_columns, format}; - } - - auto compression = chooseCompressionMethod(path_with_info.path, compression_method); - auto impl = std::make_unique(uri_without_path, path_with_info.path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); - if (!getContext()->getSettingsRef().hdfs_skip_empty_files || !impl->eof()) - { - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - return {wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)), std::nullopt, format}; - } - } - } - - void setNumRowsToLastFile(size_t num_rows) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs) - return; - - String source = uri_without_path + paths_with_info[current_index - 1].path; - auto key = getKeyForSchemaCache(source, *format, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addNumRows(key, num_rows); - } - - void setSchemaToLastFile(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) - return; - - String source = uri_without_path + paths_with_info[current_index - 1].path; - auto key = getKeyForSchemaCache(source, *format, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addColumns(key, columns); - } - - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) - return; - - Strings sources; - sources.reserve(paths_with_info.size()); - std::transform(paths_with_info.begin(), paths_with_info.end(), std::back_inserter(sources), [&](const StorageHDFS::PathWithInfo & path_with_info){ return uri_without_path + path_with_info.path; }); - auto cache_keys = getKeysForSchemaCache(sources, *format, {}, getContext()); - StorageHDFS::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); - } - - void setFormatName(const String & format_name) override - { - format = format_name; - } - - String getLastFileName() const override - { - if (current_index != 0) - return paths_with_info[current_index - 1].path; - - return ""; - } - - bool supportsLastReadBufferRecreation() const override { return true; } - - std::unique_ptr recreateLastReadBuffer() override - { - chassert(current_index > 0 && current_index <= paths_with_info.size()); - auto path_with_info = paths_with_info[current_index - 1]; - auto compression = chooseCompressionMethod(path_with_info.path, compression_method); - auto impl = std::make_unique(uri_without_path, path_with_info.path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - return wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); - } - - private: - std::optional tryGetColumnsFromCache(const std::vector & paths_with_info_) - { - auto context = getContext(); - - if (!context->getSettingsRef().schema_inference_use_cache_for_hdfs) - return std::nullopt; - - auto & schema_cache = StorageHDFS::getSchemaCache(context); - for (const auto & path_with_info : paths_with_info_) - { - auto get_last_mod_time = [&]() -> std::optional - { - if (path_with_info.info) - return path_with_info.info->last_mod_time; - - auto builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_with_info.path.c_str())); - if (hdfs_info) - return hdfs_info->mLastMod; - - return std::nullopt; - }; - - String url = uri_without_path + path_with_info.path; - if (format) - { - auto cache_key = getKeyForSchemaCache(url, *format, {}, context); - if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) - return columns; - } - else - { - /// If format is unknown, we can iterate through all possible input formats - /// and check if we have an entry with this format and this file in schema cache. - /// If we have such entry for some format, we can use this format to read the file. - for (const auto & format_name : FormatFactory::instance().getAllInputFormats()) - { - auto cache_key = getKeyForSchemaCache(url, format_name, {}, context); - if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) - { - /// Now format is known. It should be the same for all files. - format = format_name; - return columns; - } - } - } - } - - return std::nullopt; - } - - const std::vector & paths_with_info; - const String & uri_without_path; - std::optional format; - const String & compression_method; - size_t current_index = 0; - }; -} - -std::pair StorageHDFS::getTableStructureAndFormatFromDataImpl( - std::optional format, - const String & uri, - const String & compression_method, - const ContextPtr & ctx) -{ - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - auto paths_with_info = getPathsList(path_from_uri, uri, ctx); - - if (paths_with_info.empty() && (!format || !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(*format))) - { - if (format) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The table structure cannot be extracted from a {} format file, because there are no files in HDFS with provided path." - " You can specify table structure manually", *format); - - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The data format cannot be detected by the contents of the files, because there are no files in HDFS with provided path." - " You can specify the format manually"); - } - - ReadBufferIterator read_buffer_iterator(paths_with_info, uri_without_path, format, compression_method, ctx); - if (format) - return {readSchemaFromFormat(*format, std::nullopt, read_buffer_iterator, ctx), *format}; - return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, ctx); -} - -std::pair StorageHDFS::getTableStructureAndFormatFromData(const String & uri, const String & compression_method, const ContextPtr & ctx) -{ - return getTableStructureAndFormatFromDataImpl(std::nullopt, uri, compression_method, ctx); -} - -ColumnsDescription StorageHDFS::getTableStructureFromData(const String & format, const String & uri, const String & compression_method, const DB::ContextPtr & ctx) -{ - return getTableStructureAndFormatFromDataImpl(format, uri, compression_method, ctx).first; -} - -class HDFSSource::DisclosedGlobIterator::Impl -{ -public: - Impl(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - { - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - uris = getPathsList(path_from_uri, uri_without_path, context); - ActionsDAGPtr filter_dag; - if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - - if (filter_dag) - { - std::vector paths; - paths.reserve(uris.size()); - for (const auto & path_with_info : uris) - paths.push_back(path_with_info.path); - - VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context); - } - auto file_progress_callback = context->getFileProgressCallback(); - - for (auto & elem : uris) - { - elem.path = uri_without_path + elem.path; - if (file_progress_callback && elem.info) - file_progress_callback(FileProgress(0, elem.info->size)); - } - uris_iter = uris.begin(); - } - - StorageHDFS::PathWithInfo next() - { - std::lock_guard lock(mutex); - if (uris_iter != uris.end()) - { - auto answer = *uris_iter; - ++uris_iter; - return answer; - } - return {}; - } -private: - std::mutex mutex; - std::vector uris; - std::vector::iterator uris_iter; -}; - -class HDFSSource::URISIterator::Impl : WithContext -{ -public: - explicit Impl(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context_) - : WithContext(context_), uris(uris_), file_progress_callback(context_->getFileProgressCallback()) - { - ActionsDAGPtr filter_dag; - if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - - if (filter_dag) - { - std::vector paths; - paths.reserve(uris.size()); - for (const auto & uri : uris) - paths.push_back(getPathFromUriAndUriWithoutPath(uri).first); - - VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, getContext()); - } - - if (!uris.empty()) - { - auto path_and_uri = getPathFromUriAndUriWithoutPath(uris[0]); - builder = createHDFSBuilder(path_and_uri.second + "/", getContext()->getGlobalContext()->getConfigRef()); - fs = createHDFSFS(builder.get()); - } - } - - StorageHDFS::PathWithInfo next() - { - String uri; - HDFSFileInfoPtr hdfs_info; - do - { - size_t current_index = index.fetch_add(1); - if (current_index >= uris.size()) - return {"", {}}; - - uri = uris[current_index]; - auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); - hdfs_info.reset(hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str())); - } - /// Skip non-existed files. - while (!hdfs_info && String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); - - std::optional info; - if (hdfs_info) - { - info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; - if (file_progress_callback) - file_progress_callback(FileProgress(0, hdfs_info->mSize)); - } - - return {uri, info}; - } - -private: - std::atomic_size_t index = 0; - Strings uris; - HDFSBuilderWrapper builder; - HDFSFSPtr fs; - std::function file_progress_callback; -}; - -HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uri, predicate, virtual_columns, context)) {} - -StorageHDFS::PathWithInfo HDFSSource::DisclosedGlobIterator::next() -{ - return pimpl->next(); -} - -HDFSSource::URISIterator::URISIterator(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uris_, predicate, virtual_columns, context)) -{ -} - -StorageHDFS::PathWithInfo HDFSSource::URISIterator::next() -{ - return pimpl->next(); -} - -HDFSSource::HDFSSource( - const ReadFromFormatInfo & info, - StorageHDFSPtr storage_, - const ContextPtr & context_, - UInt64 max_block_size_, - std::shared_ptr file_iterator_, - bool need_only_count_) - : ISource(info.source_header, false) - , WithContext(context_) - , storage(std::move(storage_)) - , block_for_format(info.format_header) - , requested_columns(info.requested_columns) - , requested_virtual_columns(info.requested_virtual_columns) - , max_block_size(max_block_size_) - , file_iterator(file_iterator_) - , columns_description(info.columns_description) - , need_only_count(need_only_count_) -{ - initialize(); -} - -HDFSSource::~HDFSSource() = default; - -bool HDFSSource::initialize() -{ - bool skip_empty_files = getContext()->getSettingsRef().hdfs_skip_empty_files; - StorageHDFS::PathWithInfo path_with_info; - while (true) - { - path_with_info = (*file_iterator)(); - if (path_with_info.path.empty()) - return false; - - if (path_with_info.info && skip_empty_files && path_with_info.info->size == 0) - continue; - - current_path = path_with_info.path; - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_path); - - std::optional file_size; - if (!path_with_info.info) - { - auto builder = createHDFSBuilder(uri_without_path + "/", getContext()->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_from_uri.c_str())); - if (hdfs_info) - path_with_info.info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; - } - - if (path_with_info.info) - file_size = path_with_info.info->size; - - auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); - auto impl = std::make_unique( - uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings(), 0, false, file_size); - if (!skip_empty_files || !impl->eof()) - { - impl->setProgressCallback(getContext()); - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - read_buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); - break; - } - } - - current_path = path_with_info.path; - current_file_size = path_with_info.info ? std::optional(path_with_info.info->size) : std::nullopt; - - QueryPipelineBuilder builder; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(path_with_info) : std::nullopt; - if (num_rows_from_cache) - { - /// We should not return single chunk with all number of rows, - /// because there is a chance that this chunk will be materialized later - /// (it can cause memory problems even with default values in columns or when virtual columns are requested). - /// Instead, we use a special ConstChunkGenerator that will generate chunks - /// with max_block_size rows until total number of rows is reached. - auto source = std::make_shared(block_for_format, *num_rows_from_cache, max_block_size); - builder.init(Pipe(source)); - } - else - { - std::optional max_parsing_threads; - if (need_only_count) - max_parsing_threads = 1; - - input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, std::nullopt, max_parsing_threads); - - if (need_only_count) - input_format->needOnlyCount(); - - builder.init(Pipe(input_format)); - if (columns_description.hasDefaults()) - { - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, columns_description, *input_format, getContext()); - }); - } - } - - /// Add ExtractColumnsTransform to extract requested columns/subcolumns - /// from the chunk read by IInputFormat. - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, requested_columns); - }); - - pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - reader = std::make_unique(*pipeline); - - ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - return true; -} - -String HDFSSource::getName() const -{ - return "HDFSSource"; -} - -Chunk HDFSSource::generate() -{ - while (true) - { - if (isCancelled() || !reader) - { - if (reader) - reader->cancel(); - break; - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - total_rows_in_file += num_rows; - size_t chunk_size = 0; - if (input_format) - chunk_size = input_format->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, current_path, current_file_size); - return chunk; - } - - if (input_format && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(current_path, total_rows_in_file); - - total_rows_in_file = 0; - - reader.reset(); - pipeline.reset(); - input_format.reset(); - read_buf.reset(); - - if (!initialize()) - break; - } - return {}; -} - -void HDFSSource::addNumRowsToCache(const String & path, size_t num_rows) -{ - auto cache_key = getKeyForSchemaCache(path, storage->format_name, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); -} - -std::optional HDFSSource::tryGetNumRowsFromCache(const StorageHDFS::PathWithInfo & path_with_info) -{ - auto cache_key = getKeyForSchemaCache(path_with_info.path, storage->format_name, std::nullopt, getContext()); - auto get_last_mod_time = [&]() -> std::optional - { - if (path_with_info.info) - return path_with_info.info->last_mod_time; - return std::nullopt; - }; - - return StorageHDFS::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); -} - -class HDFSSink : public SinkToStorage -{ -public: - HDFSSink(const String & uri, - const String & format, - const Block & sample_block, - const ContextPtr & context, - const CompressionMethod compression_method) - : SinkToStorage(sample_block) - { - const auto & settings = context->getSettingsRef(); - write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique( - uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication, context->getWriteSettings()), - compression_method, - static_cast(settings.output_format_compression_level), - static_cast(settings.output_format_compression_zstd_window_log)); - writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); - } - - String getName() const override { return "HDFSSink"; } - - void consume(Chunk chunk) override - { - std::lock_guard lock(cancel_mutex); - if (cancelled) - return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); - } - - void onCancel() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - cancelled = true; - } - - void onException(std::exception_ptr exception) override - { - std::lock_guard lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization - release(); - } - } - - void onFinish() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - } - -private: - void finalize() - { - if (!writer) - return; - - try - { - writer->finalize(); - writer->flush(); - write_buf->sync(); - write_buf->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - release(); - throw; - } - } - - void release() - { - writer.reset(); - write_buf->finalize(); - } - - std::unique_ptr write_buf; - OutputFormatPtr writer; - std::mutex cancel_mutex; - bool cancelled = false; -}; - -namespace -{ - std::optional checkAndGetNewFileOnInsertIfNeeded(const ContextPtr & context, const String & uri, size_t sequence_number) - { - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - if (context->getSettingsRef().hdfs_truncate_on_insert || hdfsExists(fs.get(), path_from_uri.c_str())) - return std::nullopt; - - if (context->getSettingsRef().hdfs_create_new_file_on_insert) - { - auto pos = uri.find_first_of('.', uri.find_last_of('/')); - String new_uri; - do - { - new_uri = uri.substr(0, pos) + "." + std::to_string(sequence_number) + (pos == std::string::npos ? "" : uri.substr(pos)); - ++sequence_number; - } - while (!hdfsExists(fs.get(), new_uri.c_str())); - - return new_uri; - } - - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "File with path {} already exists. If you want to overwrite it, enable setting hdfs_truncate_on_insert, " - "if you want to create new file on each insert, enable setting hdfs_create_new_file_on_insert", - path_from_uri); - } -} - -class PartitionedHDFSSink : public PartitionedSink -{ -public: - PartitionedHDFSSink( - const ASTPtr & partition_by, - const String & uri_, - const String & format_, - const Block & sample_block_, - ContextPtr context_, - const CompressionMethod compression_method_) - : PartitionedSink(partition_by, context_, sample_block_) - , uri(uri_) - , format(format_) - , sample_block(sample_block_) - , context(context_) - , compression_method(compression_method_) - { - } - - SinkPtr createSinkForPartition(const String & partition_id) override - { - auto path = PartitionedSink::replaceWildcards(uri, partition_id); - PartitionedSink::validatePartitionKey(path, true); - if (auto new_path = checkAndGetNewFileOnInsertIfNeeded(context, path, 1)) - path = *new_path; - return std::make_shared(path, format, sample_block, context, compression_method); - } - -private: - const String uri; - const String format; - const Block sample_block; - ContextPtr context; - const CompressionMethod compression_method; -}; - - -bool StorageHDFS::supportsSubsetOfColumns(const ContextPtr & context_) const -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context_); -} - -class ReadFromHDFS : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromHDFS"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - void applyFilters(ActionDAGNodes added_filter_nodes) override; - - ReadFromHDFS( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - ReadFromFormatInfo info_, - bool need_only_count_, - std::shared_ptr storage_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter( - DataStream{.header = std::move(sample_block)}, - column_names_, - query_info_, - storage_snapshot_, - context_) - , info(std::move(info_)) - , need_only_count(need_only_count_) - , storage(std::move(storage_)) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - { - } - -private: - ReadFromFormatInfo info; - const bool need_only_count; - std::shared_ptr storage; - - size_t max_block_size; - size_t num_streams; - - std::shared_ptr iterator_wrapper; - - void createIterator(const ActionsDAG::Node * predicate); -}; - -void ReadFromHDFS::applyFilters(ActionDAGNodes added_filter_nodes) -{ - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - createIterator(predicate); -} - -void StorageHDFS::read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context_, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t num_streams) -{ - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_)); - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && context_->getSettingsRef().optimize_count_from_files; - - auto this_ptr = std::static_pointer_cast(shared_from_this()); - - auto reading = std::make_unique( - column_names, - query_info, - storage_snapshot, - context_, - read_from_format_info.source_header, - std::move(read_from_format_info), - need_only_count, - std::move(this_ptr), - max_block_size, - num_streams); - - query_plan.addStep(std::move(reading)); -} - -void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate) -{ - if (iterator_wrapper) - return; - - if (storage->distributed_processing) - { - iterator_wrapper = std::make_shared( - [callback = context->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo { - return StorageHDFS::PathWithInfo{callback(), std::nullopt}; - }); - } - else if (storage->is_path_with_globs) - { - /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(storage->uris[0], predicate, storage->getVirtualsList(), context); - iterator_wrapper = std::make_shared([glob_iterator]() - { - return glob_iterator->next(); - }); - } - else - { - auto uris_iterator = std::make_shared(storage->uris, predicate, storage->getVirtualsList(), context); - iterator_wrapper = std::make_shared([uris_iterator]() - { - return uris_iterator->next(); - }); - } -} - -void ReadFromHDFS::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - createIterator(nullptr); - - Pipes pipes; - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - info, - storage, - context, - max_block_size, - iterator_wrapper, - need_only_count)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); -} - -SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_, bool /*async_insert*/) -{ - String current_uri = uris.front(); - - bool has_wildcards = current_uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; - const auto * insert_query = dynamic_cast(query.get()); - auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && has_wildcards; - - if (is_partitioned_implementation) - { - String path = current_uri.substr(current_uri.find('/', current_uri.find("//") + 2)); - if (PartitionedSink::replaceWildcards(path, "").find_first_of("*?{") != std::string::npos) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back()); - - return std::make_shared( - partition_by_ast, - current_uri, - format_name, - metadata_snapshot->getSampleBlock(), - context_, - chooseCompressionMethod(current_uri, compression_method)); - } - else - { - if (is_path_with_globs) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back()); - - if (auto new_uri = checkAndGetNewFileOnInsertIfNeeded(context_, uris.front(), uris.size())) - { - uris.push_back(*new_uri); - current_uri = *new_uri; - } - - return std::make_shared(current_uri, - format_name, - metadata_snapshot->getSampleBlock(), - context_, - chooseCompressionMethod(current_uri, compression_method)); - } -} - -void StorageHDFS::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) -{ - const size_t begin_of_path = uris[0].find('/', uris[0].find("//") + 2); - const String url = uris[0].substr(0, begin_of_path); - - HDFSBuilderWrapper builder = createHDFSBuilder(url + "/", local_context->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - - for (const auto & uri : uris) - { - const String path = uri.substr(begin_of_path); - int ret = hdfsDelete(fs.get(), path.data(), 0); - if (ret) - throw Exception(ErrorCodes::ACCESS_DENIED, "Unable to truncate hdfs table: {}", std::string(hdfsGetLastError())); - } -} - - -void registerStorageHDFS(StorageFactory & factory) -{ - factory.registerStorage("HDFS", [](const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - - if (engine_args.empty() || engine_args.size() > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage HDFS requires 1, 2 or 3 arguments: " - "url, name of used format (taken from file extension by default) and optional compression method."); - - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.getLocalContext()); - - String url = checkAndGetLiteralArgument(engine_args[0], "url"); - - String format_name = "auto"; - if (engine_args.size() > 1) - { - engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.getLocalContext()); - format_name = checkAndGetLiteralArgument(engine_args[1], "format_name"); - } - - if (format_name == "auto") - format_name = FormatFactory::instance().tryGetFormatFromFileName(url).value_or("auto"); - - String compression_method; - if (engine_args.size() == 3) - { - engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.getLocalContext()); - compression_method = checkAndGetLiteralArgument(engine_args[2], "compression_method"); - } else compression_method = "auto"; - - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - - return std::make_shared( - url, args.table_id, format_name, args.columns, args.constraints, args.comment, args.getContext(), compression_method, false, partition_by); - }, - { - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::HDFS, - }); -} - -SchemaCache & StorageHDFS::getSchemaCache(const ContextPtr & ctx) -{ - static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_hdfs", DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; -} - -} - -#endif diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h deleted file mode 100644 index 9a6e192d49b..00000000000 --- a/src/Storages/HDFS/StorageHDFS.h +++ /dev/null @@ -1,190 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_HDFS - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class IInputFormat; - -/** - * This class represents table engine for external hdfs files. - * Read method is supported for now. - */ -class StorageHDFS final : public IStorage, WithContext -{ -public: - struct PathInfo - { - time_t last_mod_time; - size_t size; - }; - - struct PathWithInfo - { - PathWithInfo() = default; - PathWithInfo(const String & path_, const std::optional & info_) : path(path_), info(info_) {} - String path; - std::optional info; - }; - - StorageHDFS( - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - const ContextPtr & context_, - const String & compression_method_ = "", - bool distributed_processing_ = false, - ASTPtr partition_by = nullptr); - - String getName() const override { return "HDFS"; } - - void read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams) override; - - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; - - void truncate( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr local_context, - TableExclusiveLockHolder &) override; - - bool supportsPartitionBy() const override { return true; } - - /// Check if the format is column-oriented. - /// Is is useful because column oriented formats could effectively skip unknown columns - /// So we can create a header of only required columns in read method and ask - /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. - bool supportsSubsetOfColumns(const ContextPtr & context_) const; - - bool supportsSubcolumns() const override { return true; } - - bool supportsDynamicSubcolumns() const override { return true; } - - static ColumnsDescription getTableStructureFromData( - const String & format, - const String & uri, - const String & compression_method, - const ContextPtr & ctx); - - static std::pair getTableStructureAndFormatFromData( - const String & uri, - const String & compression_method, - const ContextPtr & ctx); - - static SchemaCache & getSchemaCache(const ContextPtr & ctx); - - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } - -protected: - friend class HDFSSource; - friend class ReadFromHDFS; - -private: - static std::pair getTableStructureAndFormatFromDataImpl( - std::optional format, - const String & uri, - const String & compression_method, - const ContextPtr & ctx); - - std::vector uris; - String format_name; - String compression_method; - const bool distributed_processing; - ASTPtr partition_by; - bool is_path_with_globs; - - LoggerPtr log = getLogger("StorageHDFS"); -}; - -class PullingPipelineExecutor; - -class HDFSSource : public ISource, WithContext -{ -public: - class DisclosedGlobIterator - { - public: - DisclosedGlobIterator(const String & uri_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); - StorageHDFS::PathWithInfo next(); - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - class URISIterator - { - public: - URISIterator(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); - StorageHDFS::PathWithInfo next(); - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - using IteratorWrapper = std::function; - using StorageHDFSPtr = std::shared_ptr; - - HDFSSource( - const ReadFromFormatInfo & info, - StorageHDFSPtr storage_, - const ContextPtr & context_, - UInt64 max_block_size_, - std::shared_ptr file_iterator_, - bool need_only_count_); - - ~HDFSSource() override; - - String getName() const override; - - Chunk generate() override; - -private: - void addNumRowsToCache(const String & path, size_t num_rows); - std::optional tryGetNumRowsFromCache(const StorageHDFS::PathWithInfo & path_with_info); - - StorageHDFSPtr storage; - Block block_for_format; - NamesAndTypesList requested_columns; - NamesAndTypesList requested_virtual_columns; - UInt64 max_block_size; - std::shared_ptr file_iterator; - ColumnsDescription columns_description; - bool need_only_count; - size_t total_rows_in_file = 0; - - std::unique_ptr read_buf; - std::shared_ptr input_format; - std::unique_ptr pipeline; - std::unique_ptr reader; - String current_path; - std::optional current_file_size; - - /// Recreate ReadBuffer and PullingPipelineExecutor for each file. - bool initialize(); -}; -} - -#endif diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp deleted file mode 100644 index bde8b84e349..00000000000 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ /dev/null @@ -1,98 +0,0 @@ -#include "config.h" -#include "Interpreters/Context_fwd.h" - -#if USE_HDFS - -#include - -#include -#include -#include -#include -#include - -#include - -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -StorageHDFSCluster::StorageHDFSCluster( - ContextPtr context_, - const String & cluster_name_, - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & compression_method) - : IStorageCluster(cluster_name_, table_id_, getLogger("StorageHDFSCluster (" + table_id_.table_name + ")")) - , uri(uri_) - , format_name(format_name_) -{ - checkHDFSURL(uri_); - context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); - - StorageInMemoryMetadata storage_metadata; - - if (columns_.empty()) - { - ColumnsDescription columns; - if (format_name == "auto") - std::tie(columns, format_name) = StorageHDFS::getTableStructureAndFormatFromData(uri_, compression_method, context_); - else - columns = StorageHDFS::getTableStructureFromData(format_name, uri_, compression_method, context_); - storage_metadata.setColumns(columns); - } - else - { - if (format_name == "auto") - format_name = StorageHDFS::getTableStructureAndFormatFromData(uri_, compression_method, context_).second; - - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); -} - -void StorageHDFSCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) -{ - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function hdfsCluster, got '{}'", queryToString(query)); - - TableFunctionHDFSCluster::updateStructureAndFormatArgumentsIfNeeded( - expression_list->children, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), format_name, context); -} - - -RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const -{ - auto iterator = std::make_shared(uri, predicate, getVirtualsList(), context); - auto callback = std::make_shared>([iter = std::move(iterator)]() mutable -> String { return iter->next().path; }); - return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; -} - -} - -#endif diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h deleted file mode 100644 index 7a651805982..00000000000 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ /dev/null @@ -1,53 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_HDFS - -#include -#include - -#include -#include -#include -#include - -namespace DB -{ - -class Context; - -class StorageHDFSCluster : public IStorageCluster -{ -public: - StorageHDFSCluster( - ContextPtr context_, - const String & cluster_name_, - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & compression_method); - - std::string getName() const override { return "HDFSCluster"; } - - RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; - - bool supportsSubcolumns() const override { return true; } - - bool supportsDynamicSubcolumns() const override { return true; } - - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } - -private: - void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override; - - String uri; - String format_name; -}; - - -} - -#endif diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index 0f9d3364ffd..81c167165d3 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 536214e159f..2c8a2e020a9 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include namespace orc { diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index b80bf8d7f46..28d8128e052 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -38,8 +38,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 0fc1e3ff8d9..8a457dd6e01 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 920155bf689..adada5c15ba 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -27,11 +27,14 @@ namespace ErrorCodes extern const int CANNOT_RESTORE_TABLE; } -IStorage::IStorage(StorageID storage_id_) +IStorage::IStorage(StorageID storage_id_, std::unique_ptr metadata_) : storage_id(std::move(storage_id_)) - , metadata(std::make_unique()) , virtuals(std::make_unique()) { + if (metadata_) + metadata.set(std::move(metadata_)); + else + metadata.set(std::make_unique()); } bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6f5c432e187..86b391bc6ac 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -99,7 +99,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo public: IStorage() = delete; /// Storage metadata can be set separately in setInMemoryMetadata method - explicit IStorage(StorageID storage_id_); + explicit IStorage(StorageID storage_id_, std::unique_ptr metadata_ = nullptr); IStorage(const IStorage &) = delete; IStorage & operator=(const IStorage &) = delete; @@ -261,6 +261,9 @@ public: /// Return true if storage can execute lightweight delete mutations. virtual bool supportsLightweightDelete() const { return false; } + /// Return true if storage has any projection. + virtual bool hasProjection() const { return false; } + /// Return true if storage can execute 'DELETE FROM' mutations. This is different from lightweight delete /// because those are internally translated into 'ALTER UDPATE' mutations. virtual bool supportsDelete() const { return false; } diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp index 56a4378cf9a..0a69bf1109f 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp @@ -93,7 +93,6 @@ String BackgroundJobsAssignee::toString(Type type) case Type::Moving: return "Moving"; } - UNREACHABLE(); } void BackgroundJobsAssignee::start() diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index db5b5a8687b..a337bdbe1c5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -43,7 +43,6 @@ class IReservation; using ReservationPtr = std::unique_ptr; class IMergeTreeReader; -class IMergeTreeDataPartWriter; class MarkCache; class UncompressedCache; class MergeTreeTransaction; @@ -74,7 +73,6 @@ public: using VirtualFields = std::unordered_map; using MergeTreeReaderPtr = std::unique_ptr; - using MergeTreeWriterPtr = std::unique_ptr; using ColumnSizeByName = std::unordered_map; using NameToNumber = std::unordered_map; @@ -106,15 +104,6 @@ public: const ValueSizeMap & avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_) const = 0; - virtual MergeTreeWriterPtr getWriter( - const NamesAndTypesList & columns_list, - const StorageMetadataPtr & metadata_snapshot, - const std::vector & indices_to_recalc, - const Statistics & stats_to_recalc_, - const CompressionCodecPtr & default_codec_, - const MergeTreeWriterSettings & writer_settings, - const MergeTreeIndexGranularity & computed_index_granularity) = 0; - virtual bool isStoredOnDisk() const = 0; virtual bool isStoredOnRemoteDisk() const = 0; @@ -172,6 +161,8 @@ public: const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; } + const SerializationByName & getSerializations() const { return serializations; } + SerializationPtr getSerialization(const String & column_name) const; SerializationPtr tryGetSerialization(const String & column_name) const; @@ -201,6 +192,7 @@ public: /// take place, you must take original name of column for this part from /// storage and pass it to this method. std::optional getColumnPosition(const String & column_name) const; + const NameToNumber & getColumnPositions() const { return column_name_to_position; } /// Returns the name of a column with minimum compressed size (as returned by getColumnSize()). /// If no checksums are present returns the name of the first physically existing column. @@ -446,6 +438,8 @@ public: bool hasProjection(const String & projection_name) const { return projection_parts.contains(projection_name); } + bool hasProjection() const { return !projection_parts.empty(); } + bool hasBrokenProjection(const String & projection_name) const; /// Return true, if all projections were loaded successfully and none was marked as broken. diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 2488c63e309..891ba1b9660 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -3,6 +3,13 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int NO_SUCH_COLUMN_IN_TABLE; +} + + Block getBlockAndPermute(const Block & block, const Names & names, const IColumn::Permutation * permutation) { Block result; @@ -38,18 +45,27 @@ Block permuteBlockIfNeeded(const Block & block, const IColumn::Permutation * per } IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( - const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr & virtual_columns_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : data_part(data_part_) - , storage(data_part_->storage) + : data_part_name(data_part_name_) + , serializations(serializations_) + , index_granularity_info(index_granularity_info_) + , storage_settings(storage_settings_) , metadata_snapshot(metadata_snapshot_) + , virtual_columns(virtual_columns_) , columns_list(columns_list_) , settings(settings_) - , index_granularity(index_granularity_) , with_final_mark(settings.can_use_adaptive_granularity) + , data_part_storage(data_part_storage_) + , index_granularity(index_granularity_) { } @@ -60,6 +76,102 @@ Columns IMergeTreeDataPartWriter::releaseIndexColumns() std::make_move_iterator(index_columns.end())); } +SerializationPtr IMergeTreeDataPartWriter::getSerialization(const String & column_name) const +{ + auto it = serializations.find(column_name); + if (it == serializations.end()) + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column or subcolumn {} in part {}", column_name, data_part_name); + + return it->second; +} + +ASTPtr IMergeTreeDataPartWriter::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const +{ + auto get_codec_or_default = [&](const auto & column_desc) + { + return column_desc.codec ? column_desc.codec : default_codec->getFullCodecDesc(); + }; + + const auto & columns = metadata_snapshot->getColumns(); + if (const auto * column_desc = columns.tryGet(column_name)) + return get_codec_or_default(*column_desc); + + if (const auto * virtual_desc = virtual_columns->tryGetDescription(column_name)) + return get_codec_or_default(*virtual_desc); + + return default_codec->getFullCodecDesc(); +} + + IMergeTreeDataPartWriter::~IMergeTreeDataPartWriter() = default; + +MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, + const ColumnPositions & column_positions, + const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr & virtual_columns, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension_, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity); + +MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr & virtual_columns, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension_, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity); + + +MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( + MergeTreeDataPartType part_type, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, + const ColumnPositions & column_positions, + const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr & virtual_columns, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension_, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity) +{ + if (part_type == MergeTreeDataPartType::Compact) + return createMergeTreeDataPartCompactWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, column_positions, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); + else if (part_type == MergeTreeDataPartType::Wide) + return createMergeTreeDataPartWideWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown part type: {}", part_type.toString()); +} + } diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 3f359904ddd..f04beb37ebb 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -1,12 +1,13 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -22,9 +23,14 @@ class IMergeTreeDataPartWriter : private boost::noncopyable { public: IMergeTreeDataPartWriter( - const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr & virtual_columns_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_ = {}); @@ -32,7 +38,7 @@ public: virtual void write(const Block & block, const IColumn::Permutation * permutation) = 0; - virtual void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) = 0; + virtual void fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) = 0; virtual void finish(bool sync) = 0; @@ -40,16 +46,48 @@ public: const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; } protected: + SerializationPtr getSerialization(const String & column_name) const; - const MergeTreeMutableDataPartPtr data_part; - const MergeTreeData & storage; + ASTPtr getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; + + IDataPartStorage & getDataPartStorage() { return *data_part_storage; } + + const String data_part_name; + /// Serializations for every columns and subcolumns by their names. + const SerializationByName serializations; + const MergeTreeIndexGranularityInfo index_granularity_info; + const MergeTreeSettingsPtr storage_settings; const StorageMetadataPtr metadata_snapshot; + const VirtualsDescriptionPtr virtual_columns; const NamesAndTypesList columns_list; const MergeTreeWriterSettings settings; - MergeTreeIndexGranularity index_granularity; const bool with_final_mark; + MutableDataPartStoragePtr data_part_storage; MutableColumns index_columns; + MergeTreeIndexGranularity index_granularity; }; +using MergeTreeDataPartWriterPtr = std::unique_ptr; +using ColumnPositions = std::unordered_map; + +MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( + MergeTreeDataPartType part_type, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, + const ColumnPositions & column_positions, + const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr & virtual_columns_, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity); + } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index c8d6aa0ba65..89c813ab233 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -7,20 +7,21 @@ namespace DB { IMergedBlockOutputStream::IMergedBlockOutputStream( - const MergeTreeMutableDataPartPtr & data_part, + const MergeTreeSettingsPtr & storage_settings_, + MutableDataPartStoragePtr data_part_storage_, const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list, bool reset_columns_) - : storage(data_part->storage) + : storage_settings(storage_settings_) , metadata_snapshot(metadata_snapshot_) - , data_part_storage(data_part->getDataPartStoragePtr()) + , data_part_storage(data_part_storage_) , reset_columns(reset_columns_) { if (reset_columns) { SerializationInfo::Settings info_settings = { - .ratio_of_defaults_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization, + .ratio_of_defaults_for_sparse = storage_settings->ratio_of_defaults_for_sparse_serialization, .choose_kind = false, }; @@ -42,7 +43,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( return {}; for (const auto & column : empty_columns) - LOG_TRACE(storage.log, "Skipping expired/empty column {} for part {}", column, data_part->name); + LOG_TRACE(data_part->storage.log, "Skipping expired/empty column {} for part {}", column, data_part->name); /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. std::map stream_counts; @@ -91,7 +92,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( } else /// If we have no file in checksums it doesn't exist on disk { - LOG_TRACE(storage.log, "Files {} doesn't exist in checksums so it doesn't exist on disk, will not try to remove it", *itr); + LOG_TRACE(data_part->storage.log, "Files {} doesn't exist in checksums so it doesn't exist on disk, will not try to remove it", *itr); itr = remove_files.erase(itr); } } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index ca4e3899b29..a9b058418ea 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -1,10 +1,12 @@ #pragma once -#include "Storages/MergeTree/IDataPartStorage.h" +#include +#include #include #include #include #include +#include namespace DB { @@ -13,7 +15,8 @@ class IMergedBlockOutputStream { public: IMergedBlockOutputStream( - const MergeTreeMutableDataPartPtr & data_part, + const MergeTreeSettingsPtr & storage_settings_, + MutableDataPartStoragePtr data_part_storage_, const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list, bool reset_columns_); @@ -39,11 +42,13 @@ protected: SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums); - const MergeTreeData & storage; + MergeTreeSettingsPtr storage_settings; + LoggerPtr log; + StorageMetadataPtr metadata_snapshot; MutableDataPartStoragePtr data_part_storage; - IMergeTreeDataPart::MergeTreeWriterPtr writer; + MergeTreeDataPartWriterPtr writer; bool reset_columns = false; SerializationInfoByName new_serialization_infos; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 849240502e4..9666da574fb 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -2664,6 +2664,13 @@ BoolMask KeyCondition::checkInHyperrectangle( else if (element.function == RPNElement::FUNCTION_IN_RANGE || element.function == RPNElement::FUNCTION_NOT_IN_RANGE) { + if (element.key_column >= hyperrectangle.size()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Hyperrectangle size is {}, but requested element at posittion {} ({})", + hyperrectangle.size(), element.key_column, element.toString()); + } + const Range * key_range = &hyperrectangle[element.key_column]; /// The case when the column is wrapped in a chain of possibly monotonic functions. @@ -2957,8 +2964,6 @@ String KeyCondition::RPNElement::toString(std::string_view column_name, bool pri case ALWAYS_TRUE: return "true"; } - - UNREACHABLE(); } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index a9109832521..e43b6c615b3 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -9,7 +9,7 @@ #include #include #include - +#include #include #include #include @@ -34,6 +34,7 @@ #include #include #include +#include #include namespace DB @@ -378,7 +379,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() MergeTreeIndexFactory::instance().getMany(global_ctx->metadata_snapshot->getSecondaryIndices()), MergeTreeStatisticsFactory::instance().getMany(global_ctx->metadata_snapshot->getColumns()), ctx->compression_codec, - global_ctx->txn, + global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, ctx->blocks_are_granules_size, global_ctx->context->getWriteSettings()); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7a41961a3d0..b6373a22d9c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1177,8 +1177,6 @@ String MergeTreeData::MergingParams::getModeName() const case Graphite: return "Graphite"; case VersionedCollapsing: return "VersionedCollapsing"; } - - UNREACHABLE(); } Int64 MergeTreeData::getMaxBlockNumber() const @@ -6135,6 +6133,21 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } +bool MergeTreeData::hasProjection() const +{ + auto lock = lockParts(); + for (const auto & part : data_parts_by_info) + { + if (part->getState() == MergeTreeDataPartState::Outdated + || part->getState() == MergeTreeDataPartState::Deleting) + continue; + + if (part->hasProjection()) + return true; + } + return false; +} + MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const { ProjectionPartsVector res; @@ -8477,7 +8490,7 @@ std::pair MergeTreeData::createE MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), Statistics{}, - compression_codec, txn); + compression_codec, txn ? txn->tid : Tx::PrehistoricTID); bool sync_on_insert = settings->fsync_after_insert; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e65016221f0..fb8f2ec29aa 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -439,6 +439,8 @@ public: bool supportsLightweightDelete() const override; + bool hasProjection() const override; + bool areAsynchronousInsertsEnabled() const override { return getSettings()->async_insert; } bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 418b2d8f81b..4a160e5e229 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -47,26 +47,36 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( avg_value_size_hints, profile_callback, CLOCK_MONOTONIC_COARSE); } -IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( +MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, + const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr & virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, + const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) { NamesAndTypesList ordered_columns_list; std::copy_if(columns_list.begin(), columns_list.end(), std::back_inserter(ordered_columns_list), - [this](const auto & column) { return getColumnPosition(column.name) != std::nullopt; }); + [&column_positions](const auto & column) { return column_positions.contains(column.name); }); /// Order of writing is important in compact format - ordered_columns_list.sort([this](const auto & lhs, const auto & rhs) - { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); + ordered_columns_list.sort([&column_positions](const auto & lhs, const auto & rhs) + { return column_positions.at(lhs.name) < column_positions.at(rhs.name); }); return std::make_unique( - shared_from_this(), ordered_columns_list, metadata_snapshot, - indices_to_recalc, stats_to_recalc_, getMarksFileExtension(), + data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, ordered_columns_list, metadata_snapshot, virtual_columns, + indices_to_recalc, stats_to_recalc_, marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 3a4e7b95f33..1fb84424774 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -40,15 +40,6 @@ public: const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; - MergeTreeWriterPtr getWriter( - const NamesAndTypesList & columns_list, - const StorageMetadataPtr & metadata_snapshot, - const std::vector & indices_to_recalc, - const Statistics & stats_to_recalc_, - const CompressionCodecPtr & default_codec_, - const MergeTreeWriterSettings & writer_settings, - const MergeTreeIndexGranularity & computed_index_granularity) override; - bool isStoredOnDisk() const override { return true; } bool isStoredOnRemoteDisk() const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index fc3108e522a..149f86cef00 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -53,19 +53,28 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( profile_callback); } -IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( +MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr & virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, + const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) { return std::make_unique( - shared_from_this(), columns_list, - metadata_snapshot, indices_to_recalc, stats_to_recalc_, - getMarksFileExtension(), + data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, + metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 84eeec4211b..7465e08b7c4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -35,15 +35,6 @@ public: const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; - MergeTreeWriterPtr getWriter( - const NamesAndTypesList & columns_list, - const StorageMetadataPtr & metadata_snapshot, - const std::vector & indices_to_recalc, - const Statistics & stats_to_recalc_, - const CompressionCodecPtr & default_codec_, - const MergeTreeWriterSettings & writer_settings, - const MergeTreeIndexGranularity & computed_index_granularity) override; - bool isStoredOnDisk() const override { return true; } bool isStoredOnRemoteDisk() const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index e34822ce6df..fb0f0ba9154 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -10,32 +10,41 @@ namespace ErrorCodes } MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( - const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc_, const Statistics & stats_to_recalc, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, + : MergeTreeDataPartWriterOnDisk( + data_part_name_, logger_name_, serializations_, + data_part_storage_, index_granularity_info_, storage_settings_, + columns_list_, metadata_snapshot_, virtual_columns_, indices_to_recalc_, stats_to_recalc, marks_file_extension_, default_codec_, settings_, index_granularity_) - , plain_file(data_part_->getDataPartStorage().writeFile( + , plain_file(getDataPartStorage().writeFile( MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, settings.max_compress_block_size, settings_.query_write_settings)) , plain_hashing(*plain_file) { - marks_file = data_part_->getDataPartStorage().writeFile( + marks_file = getDataPartStorage().writeFile( MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension_, 4096, settings_.query_write_settings); marks_file_hashing = std::make_unique(*marks_file); - if (data_part_->index_granularity_info.mark_type.compressed) + if (index_granularity_info.mark_type.compressed) { marks_compressor = std::make_unique( *marks_file_hashing, @@ -45,10 +54,9 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( marks_source_hashing = std::make_unique(*marks_compressor); } - auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); for (const auto & column : columns_list) { - auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); + auto compression = getCodecDescOrDefault(column.name, default_codec); addStreams(column, nullptr, compression); } } @@ -59,12 +67,11 @@ void MergeTreeDataPartWriterCompact::initDynamicStreamsIfNeeded(const Block & bl return; is_dynamic_streams_initialized = true; - auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); for (const auto & column : columns_list) { if (column.type->hasDynamicSubcolumns()) { - auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); + auto compression = getCodecDescOrDefault(column.name, default_codec); addStreams(column, block.getByName(column.name).column, compression); } } @@ -98,7 +105,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & name_and compressed_streams.emplace(stream_name, stream); }; - data_part->getSerialization(name_and_type.name)->enumerateStreams(callback, name_and_type.type, column); + getSerialization(name_and_type.name)->enumerateStreams(callback, name_and_type.type, column); } namespace @@ -251,7 +258,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeBinaryLittleEndian(static_cast(0), marks_out); writeColumnSingleGranule( - block.getByName(name_and_type->name), data_part->getSerialization(name_and_type->name), + block.getByName(name_and_type->name), getSerialization(name_and_type->name), stream_getter, granule.start_row, granule.rows_to_write); /// Each type always have at least one substream @@ -262,7 +269,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G } } -void MergeTreeDataPartWriterCompact::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterCompact::fillDataChecksums(MergeTreeDataPartChecksums & checksums) { if (columns_buffer.size() != 0) { @@ -432,7 +439,7 @@ size_t MergeTreeDataPartWriterCompact::ColumnsBuffer::size() const return accumulated_columns.at(0)->size(); } -void MergeTreeDataPartWriterCompact::fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & /*checksums_to_remove*/) +void MergeTreeDataPartWriterCompact::fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & /*checksums_to_remove*/) { // If we don't have anything to write, skip finalization. if (!columns_list.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index f35479387f6..a5527b74e69 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -11,9 +11,15 @@ class MergeTreeDataPartWriterCompact : public MergeTreeDataPartWriterOnDisk { public: MergeTreeDataPartWriterCompact( - const MergeTreeMutableDataPartPtr & data_part, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc, const String & marks_file_extension, @@ -23,12 +29,12 @@ public: void write(const Block & block, const IColumn::Permutation * permutation) override; - void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) override; + void fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) override; void finish(bool sync) override; private: /// Finish serialization of the data. Flush rows in buffer to disk, compute checksums. - void fillDataChecksums(IMergeTreeDataPart::Checksums & checksums); + void fillDataChecksums(MergeTreeDataPartChecksums & checksums); void finishDataSerialization(bool sync); void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 491d2399b82..0a8920790e0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -140,16 +140,24 @@ void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(Merg MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( - const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr & virtual_columns_, const MergeTreeIndices & indices_to_recalc_, const Statistics & stats_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : IMergeTreeDataPartWriter(data_part_, columns_list_, metadata_snapshot_, settings_, index_granularity_) + : IMergeTreeDataPartWriter( + data_part_name_, serializations_, data_part_storage_, index_granularity_info_, + storage_settings_, columns_list_, metadata_snapshot_, virtual_columns_, settings_, index_granularity_) , skip_indices(indices_to_recalc_) , stats(stats_to_recalc_) , marks_file_extension(marks_file_extension_) @@ -157,14 +165,14 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( , compute_granularity(index_granularity.empty()) , compress_primary_key(settings.compress_primary_key) , execution_stats(skip_indices.size(), stats.size()) - , log(getLogger(storage.getLogName() + " (DataPartWriter)")) + , log(getLogger(logger_name_ + " (DataPartWriter)")) { if (settings.blocks_are_granules_size && !index_granularity.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't take information about index granularity from blocks, when non empty index_granularity array specified"); - if (!data_part->getDataPartStorage().exists()) - data_part->getDataPartStorage().createDirectories(); + if (!getDataPartStorage().exists()) + getDataPartStorage().createDirectories(); if (settings.rewrite_primary_key) initPrimaryIndex(); @@ -223,7 +231,6 @@ static size_t computeIndexGranularityImpl( size_t MergeTreeDataPartWriterOnDisk::computeIndexGranularity(const Block & block) const { - const auto storage_settings = storage.getSettings(); return computeIndexGranularityImpl( block, storage_settings->index_granularity_bytes, @@ -237,7 +244,7 @@ void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() if (metadata_snapshot->hasPrimaryKey()) { String index_name = "primary" + getIndexExtension(compress_primary_key); - index_file_stream = data_part->getDataPartStorage().writeFile(index_name, DBMS_DEFAULT_BUFFER_SIZE, settings.query_write_settings); + index_file_stream = getDataPartStorage().writeFile(index_name, DBMS_DEFAULT_BUFFER_SIZE, settings.query_write_settings); index_file_hashing_stream = std::make_unique(*index_file_stream); if (compress_primary_key) @@ -256,7 +263,7 @@ void MergeTreeDataPartWriterOnDisk::initStatistics() String stats_name = stat_ptr->getFileName(); stats_streams.emplace_back(std::make_unique>( stats_name, - data_part->getDataPartStoragePtr(), + data_part_storage, stats_name, STAT_FILE_SUFFIX, default_codec, settings.max_compress_block_size, settings.query_write_settings)); @@ -275,7 +282,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() skip_indices_streams.emplace_back( std::make_unique>( stream_name, - data_part->getDataPartStoragePtr(), + data_part_storage, stream_name, skip_index->getSerializedFileExtension(), stream_name, marks_file_extension, default_codec, settings.max_compress_block_size, @@ -285,7 +292,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() GinIndexStorePtr store = nullptr; if (typeid_cast(&*skip_index) != nullptr) { - store = std::make_shared(stream_name, data_part->getDataPartStoragePtr(), data_part->getDataPartStoragePtr(), storage.getSettings()->max_digestion_size_per_segment); + store = std::make_shared(stream_name, data_part_storage, data_part_storage, storage_settings->max_digestion_size_per_segment); gin_index_stores[stream_name] = store; } skip_indices_aggregators.push_back(skip_index->createIndexAggregatorForPart(store, settings)); @@ -498,7 +505,7 @@ void MergeTreeDataPartWriterOnDisk::finishStatisticsSerialization(bool sync) } for (size_t i = 0; i < stats.size(); ++i) - LOG_DEBUG(log, "Spent {} ms calculating statistics {} for the part {}", execution_stats.statistics_build_us[i] / 1000, stats[i]->columnName(), data_part->name); + LOG_DEBUG(log, "Spent {} ms calculating statistics {} for the part {}", execution_stats.statistics_build_us[i] / 1000, stats[i]->columnName(), data_part_name); } void MergeTreeDataPartWriterOnDisk::fillStatisticsChecksums(MergeTreeData::DataPart::Checksums & checksums) @@ -524,7 +531,7 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync) store.second->finalize(); for (size_t i = 0; i < skip_indices.size(); ++i) - LOG_DEBUG(log, "Spent {} ms calculating index {} for the part {}", execution_stats.skip_indices_build_us[i] / 1000, skip_indices[i]->index.name, data_part->name); + LOG_DEBUG(log, "Spent {} ms calculating index {} for the part {}", execution_stats.skip_indices_build_us[i] / 1000, skip_indices[i]->index.name, data_part_name); gin_index_stores.clear(); skip_indices_streams.clear(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 9f2cc3970fa..0c31cabc8c4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -5,9 +5,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -97,16 +94,22 @@ public: void sync() const; - void addToChecksums(IMergeTreeDataPart::Checksums & checksums); + void addToChecksums(MergeTreeDataPartChecksums & checksums); }; using StreamPtr = std::unique_ptr>; using StatisticStreamPtr = std::unique_ptr>; MergeTreeDataPartWriterOnDisk( - const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension, @@ -133,13 +136,13 @@ protected: void calculateAndSerializeStatistics(const Block & stats_block); /// Finishes primary index serialization: write final primary index row (if required) and compute checksums - void fillPrimaryIndexChecksums(MergeTreeData::DataPart::Checksums & checksums); + void fillPrimaryIndexChecksums(MergeTreeDataPartChecksums & checksums); void finishPrimaryIndexSerialization(bool sync); /// Finishes skip indices serialization: write all accumulated data to disk and compute checksums - void fillSkipIndicesChecksums(MergeTreeData::DataPart::Checksums & checksums); + void fillSkipIndicesChecksums(MergeTreeDataPartChecksums & checksums); void finishSkipIndicesSerialization(bool sync); - void fillStatisticsChecksums(MergeTreeData::DataPart::Checksums & checksums); + void fillStatisticsChecksums(MergeTreeDataPartChecksums & checksums); void finishStatisticsSerialization(bool sync); /// Get global number of the current which we are writing (or going to start to write) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index fb7ee9f7fe8..afa14d8a98a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -76,23 +76,31 @@ Granules getGranulesToWrite(const MergeTreeIndexGranularity & index_granularity, } MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( - const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc_, const Statistics & stats_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, - indices_to_recalc_, stats_to_recalc_, marks_file_extension_, - default_codec_, settings_, index_granularity_) + : MergeTreeDataPartWriterOnDisk( + data_part_name_, logger_name_, serializations_, + data_part_storage_, index_granularity_info_, storage_settings_, + columns_list_, metadata_snapshot_, virtual_columns_, + indices_to_recalc_, stats_to_recalc_, marks_file_extension_, + default_codec_, settings_, index_granularity_) { - auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); for (const auto & column : columns_list) { - auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); + auto compression = getCodecDescOrDefault(column.name, default_codec); addStreams(column, nullptr, compression); } } @@ -104,12 +112,11 @@ void MergeTreeDataPartWriterWide::initDynamicStreamsIfNeeded(const DB::Block & b is_dynamic_streams_initialized = true; block_sample = block.cloneEmpty(); - auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); for (const auto & column : columns_list) { if (column.type->hasDynamicSubcolumns()) { - auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); + auto compression = getCodecDescOrDefault(column.name, default_codec); addStreams(column, block_sample.getByName(column.name).column, compression); } } @@ -124,7 +131,6 @@ void MergeTreeDataPartWriterWide::addStreams( { assert(!substream_path.empty()); - auto storage_settings = storage.getSettings(); auto full_stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path); String stream_name; @@ -168,7 +174,7 @@ void MergeTreeDataPartWriterWide::addStreams( column_streams[stream_name] = std::make_unique>( stream_name, - data_part->getDataPartStoragePtr(), + data_part_storage, stream_name, DATA_FILE_EXTENSION, stream_name, marks_file_extension, compression_codec, @@ -182,7 +188,7 @@ void MergeTreeDataPartWriterWide::addStreams( }; ISerialization::SubstreamPath path; - data_part->getSerialization(name_and_type.name)->enumerateStreams(callback, name_and_type.type, column); + getSerialization(name_and_type.name)->enumerateStreams(callback, name_and_type.type, column); } const String & MergeTreeDataPartWriterWide::getStreamName( @@ -286,7 +292,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm { auto & column = block_to_write.getByName(it->name); - if (data_part->getSerialization(it->name)->getKind() != ISerialization::Kind::SPARSE) + if (getSerialization(it->name)->getKind() != ISerialization::Kind::SPARSE) column.column = recursiveRemoveSparse(column.column); if (permutation) @@ -358,7 +364,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( min_compress_block_size = value->safeGet(); if (!min_compress_block_size) min_compress_block_size = settings.min_compress_block_size; - data_part->getSerialization(name_and_type.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + getSerialization(name_and_type.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; auto stream_name = getStreamName(name_and_type, substream_path); @@ -392,7 +398,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( ISerialization::SerializeBinaryBulkSettings & serialize_settings, const Granule & granule) { - const auto & serialization = data_part->getSerialization(name_and_type.name); + const auto & serialization = getSerialization(name_and_type.name); serialization->serializeBinaryBulkWithMultipleStreams(column, granule.start_row, granule.rows_to_write, serialize_settings, serialization_state); /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. @@ -422,7 +428,7 @@ void MergeTreeDataPartWriterWide::writeColumn( const auto & [name, type] = name_and_type; auto [it, inserted] = serialization_states.emplace(name, nullptr); - auto serialization = data_part->getSerialization(name_and_type.name); + auto serialization = getSerialization(name_and_type.name); if (inserted) { @@ -431,11 +437,10 @@ void MergeTreeDataPartWriterWide::writeColumn( serialization->serializeBinaryBulkStatePrefix(column, serialize_settings, it->second); } - const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); - serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size; + serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part; for (const auto & granule : granules) { @@ -484,7 +489,7 @@ void MergeTreeDataPartWriterWide::writeColumn( void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePair & name_type) { const auto & [name, type] = name_type; - const auto & serialization = data_part->getSerialization(name_type.name); + const auto & serialization = getSerialization(name_type.name); if (!type->isValueRepresentedByNumber() || type->haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type->getName()); @@ -494,21 +499,21 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai String bin_path = escaped_name + DATA_FILE_EXTENSION; /// Some columns may be removed because of ttl. Skip them. - if (!data_part->getDataPartStorage().exists(mrk_path)) + if (!getDataPartStorage().exists(mrk_path)) return; - auto mrk_file_in = data_part->getDataPartStorage().readFile(mrk_path, {}, std::nullopt, std::nullopt); + auto mrk_file_in = getDataPartStorage().readFile(mrk_path, {}, std::nullopt, std::nullopt); std::unique_ptr mrk_in; - if (data_part->index_granularity_info.mark_type.compressed) + if (index_granularity_info.mark_type.compressed) mrk_in = std::make_unique(std::move(mrk_file_in)); else mrk_in = std::move(mrk_file_in); - DB::CompressedReadBufferFromFile bin_in(data_part->getDataPartStorage().readFile(bin_path, {}, std::nullopt, std::nullopt)); + DB::CompressedReadBufferFromFile bin_in(getDataPartStorage().readFile(bin_path, {}, std::nullopt, std::nullopt)); bool must_be_last = false; UInt64 offset_in_compressed_file = 0; UInt64 offset_in_decompressed_block = 0; - UInt64 index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity; + UInt64 index_granularity_rows = index_granularity_info.fixed_index_granularity; size_t mark_num; @@ -524,7 +529,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai if (settings.can_use_adaptive_granularity) readBinaryLittleEndian(index_granularity_rows, *mrk_in); else - index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity; + index_granularity_rows = index_granularity_info.fixed_index_granularity; if (must_be_last) { @@ -557,7 +562,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{}" " (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}", - data_part->getDataPartStorage().getFullPath(), + getDataPartStorage().getFullPath(), mark_num, offset_in_compressed_file, offset_in_decompressed_block, index_granularity.getMarkRows(mark_num), index_granularity_rows, index_granularity.getMarksCount()); @@ -617,12 +622,11 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai } } -void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) +void MergeTreeDataPartWriterWide::fillDataChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) { - const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size; + serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part; WrittenOffsetColumns offset_columns; if (rows_written_in_last_mark > 0) { @@ -646,7 +650,7 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksum { serialize_settings.getter = createStreamGetter(*it, written_offset_columns ? *written_offset_columns : offset_columns); serialize_settings.dynamic_write_statistics = ISerialization::SerializeBinaryBulkSettings::DynamicStatisticsMode::SUFFIX; - data_part->getSerialization(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); + getSerialization(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); } if (write_final_mark) @@ -689,7 +693,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync) { if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && data_part->getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) + && getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) { validateColumnOfFixedSize(column); } @@ -698,7 +702,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync) } -void MergeTreeDataPartWriterWide::fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) +void MergeTreeDataPartWriterWide::fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) { // If we don't have anything to write, skip finalization. if (!columns_list.empty()) @@ -732,7 +736,7 @@ void MergeTreeDataPartWriterWide::writeFinalMark( { writeSingleMark(name_and_type, offset_columns, 0); /// Memoize information about offsets - data_part->getSerialization(name_and_type.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + getSerialization(name_and_type.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (is_offsets) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 8343144f2e1..9d18ac76880 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -21,9 +21,15 @@ class MergeTreeDataPartWriterWide : public MergeTreeDataPartWriterOnDisk { public: MergeTreeDataPartWriterWide( - const MergeTreeMutableDataPartPtr & data_part, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension, @@ -33,14 +39,14 @@ public: void write(const Block & block, const IColumn::Permutation * permutation) override; - void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) final; + void fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) final; void finish(bool sync) final; private: /// Finish serialization of data: write final mark if required and compute checksums /// Also validate written data in debug mode - void fillDataChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove); + void fillDataChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove); void finishDataSerialization(bool sync); /// Write data of one column. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index e5821075c3f..df4087b8546 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -360,8 +360,6 @@ Block MergeTreeDataWriter::mergeBlock( return std::make_shared( block, 1, sort_description, block_size + 1, /*block_size_bytes=*/0, merging_params.graphite_params, time(nullptr)); } - - UNREACHABLE(); }; auto merging_algorithm = get_merging_algorithm(); @@ -600,7 +598,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( indices, MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()), compression_codec, - context->getCurrentTransaction(), + context->getCurrentTransaction() ? context->getCurrentTransaction()->tid : Tx::PrehistoricTID, false, false, context->getWriteSettings()); @@ -738,7 +736,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( MergeTreeIndices{}, Statistics{}, /// TODO(hanfei): It should be helpful to write statistics for projection result. compression_codec, - NO_TRANSACTION_PTR, + Tx::PrehistoricTID, false, false, data.getContext()->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 562bfe7c439..a9125b4047e 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -74,6 +74,8 @@ struct MergeTreeWriterSettings , blocks_are_granules_size(blocks_are_granules_size_) , query_write_settings(query_write_settings_) , max_threads_for_annoy_index_creation(global_settings.max_threads_for_annoy_index_creation) + , low_cardinality_max_dictionary_size(global_settings.low_cardinality_max_dictionary_size) + , low_cardinality_use_single_dictionary_for_part(global_settings.low_cardinality_use_single_dictionary_for_part != 0) { } @@ -93,6 +95,9 @@ struct MergeTreeWriterSettings WriteSettings query_write_settings; size_t max_threads_for_annoy_index_creation; + + size_t low_cardinality_max_dictionary_size; + bool low_cardinality_use_single_dictionary_for_part; }; } diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index cd18e8d5a28..b240f80ee13 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -413,12 +413,12 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataM partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file, {}); } -std::unique_ptr MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const +std::unique_ptr MergeTreePartition::store( + StorageMetadataPtr metadata_snapshot, ContextPtr storage_context, + IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const { - auto metadata_snapshot = storage.getInMemoryMetadataPtr(); - const auto & context = storage.getContext(); - const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; - return store(partition_key_sample, data_part_storage, checksums, context->getWriteSettings()); + const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage_context).sample_block; + return store(partition_key_sample, data_part_storage, checksums, storage_context->getWriteSettings()); } std::unique_ptr MergeTreePartition::store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 78b141f26ec..44def70bdd9 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -44,7 +44,9 @@ public: /// Store functions return write buffer with written but not finalized data. /// User must call finish() for returned object. - [[nodiscard]] std::unique_ptr store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; + [[nodiscard]] std::unique_ptr store( + StorageMetadataPtr metadata_snapshot, ContextPtr storage_context, + IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; [[nodiscard]] std::unique_ptr store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const; void assign(const MergeTreePartition & other) { value = other.value; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index d8555d69788..c5799fab09f 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -21,35 +22,39 @@ MergedBlockOutputStream::MergedBlockOutputStream( const MergeTreeIndices & skip_indices, const Statistics & statistics, CompressionCodecPtr default_codec_, - const MergeTreeTransactionPtr & txn, + TransactionID tid, bool reset_columns_, bool blocks_are_granules_size, const WriteSettings & write_settings_, const MergeTreeIndexGranularity & computed_index_granularity) - : IMergedBlockOutputStream(data_part, metadata_snapshot_, columns_list_, reset_columns_) + : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, reset_columns_) , columns_list(columns_list_) , default_codec(default_codec_) , write_settings(write_settings_) { MergeTreeWriterSettings writer_settings( - storage.getContext()->getSettings(), + data_part->storage.getContext()->getSettings(), write_settings, - storage.getSettings(), + storage_settings, data_part->index_granularity_info.mark_type.adaptive, /* rewrite_primary_key = */ true, blocks_are_granules_size); + /// TODO: looks like isStoredOnDisk() is always true for MergeTreeDataPart if (data_part->isStoredOnDisk()) data_part_storage->createDirectories(); - /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; /// NOTE do not pass context for writing to system.transactions_info_log, /// because part may have temporary name (with temporary block numbers). Will write it later. data_part->version.setCreationTID(tid, nullptr); data_part->storeVersionMetadata(); - writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, statistics, default_codec, writer_settings, computed_index_granularity); + writer = createMergeTreeDataPartWriter(data_part->getType(), + data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), + data_part_storage, data_part->index_granularity_info, + storage_settings, + columns_list, data_part->getColumnPositions(), metadata_snapshot, data_part->storage.getVirtualsPtr(), + skip_indices, statistics, data_part->getMarksFileExtension(), default_codec, writer_settings, computed_index_granularity); } /// If data is pre-sorted. @@ -208,7 +213,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis if (new_part->isProjectionPart()) { - if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) + if (new_part->storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) { auto count_out = new_part->getDataPartStorage().writeFile("count.txt", 4096, write_settings); HashingWriteBuffer count_out_hashing(*count_out); @@ -234,14 +239,16 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis written_files.emplace_back(std::move(out)); } - if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + if (new_part->storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - if (auto file = new_part->partition.store(storage, new_part->getDataPartStorage(), checksums)) + if (auto file = new_part->partition.store( + new_part->storage.getInMemoryMetadataPtr(), new_part->storage.getContext(), + new_part->getDataPartStorage(), checksums)) written_files.emplace_back(std::move(file)); if (new_part->minmax_idx->initialized) { - auto files = new_part->minmax_idx->store(storage, new_part->getDataPartStorage(), checksums); + auto files = new_part->minmax_idx->store(new_part->storage, new_part->getDataPartStorage(), checksums); for (auto & file : files) written_files.emplace_back(std::move(file)); } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 540b3b3bffa..c1e3d75fefc 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -22,7 +22,7 @@ public: const MergeTreeIndices & skip_indices, const Statistics & statistics, CompressionCodecPtr default_codec_, - const MergeTreeTransactionPtr & txn, + TransactionID tid, bool reset_columns_ = false, bool blocks_are_granules_size = false, const WriteSettings & write_settings = {}, diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 728b2e38833..674a9bd498f 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -20,11 +20,10 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) - : IMergedBlockOutputStream(data_part, metadata_snapshot_, header_.getNamesAndTypesList(), /*reset_columns=*/ true) + : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, header_.getNamesAndTypesList(), /*reset_columns=*/ true) , header(header_) { const auto & global_settings = data_part->storage.getContext()->getSettings(); - const auto & storage_settings = data_part->storage.getSettings(); MergeTreeWriterSettings writer_settings( global_settings, @@ -33,11 +32,18 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( index_granularity_info ? index_granularity_info->mark_type.adaptive : data_part->storage.canUseAdaptiveGranularity(), /* rewrite_primary_key = */ false); - writer = data_part->getWriter( + writer = createMergeTreeDataPartWriter( + data_part->getType(), + data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), + data_part_storage, data_part->index_granularity_info, + storage_settings, header.getNamesAndTypesList(), + data_part->getColumnPositions(), metadata_snapshot_, + data_part->storage.getVirtualsPtr(), indices_to_recalc, stats_to_recalc_, + data_part->getMarksFileExtension(), default_codec, writer_settings, index_granularity); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 377fb5a1912..43238c5bcbc 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -1675,7 +1676,7 @@ private: skip_indices, stats_to_rewrite, ctx->compression_codec, - ctx->txn, + ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, /*blocks_are_granules_size=*/ false, ctx->context->getWriteSettings(), diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 78fcfabb704..4228d7b70b6 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -616,8 +616,6 @@ PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::st } } } - - UNREACHABLE(); } void PartMovesBetweenShardsOrchestrator::removePins(const Entry & entry, zkutil::ZooKeeperPtr zk) diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp new file mode 100644 index 00000000000..ada3e2e9323 --- /dev/null +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -0,0 +1,551 @@ +#include + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +const std::unordered_set required_configuration_keys = { + "blob_path", + "container", +}; + +const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "structure", + "compression_method", + "account_name", + "account_key", + "connection_string", + "storage_account_url", +}; + +using AzureClient = Azure::Storage::Blobs::BlobContainerClient; +using AzureClientPtr = std::unique_ptr; + +namespace +{ + bool isConnectionString(const std::string & candidate) + { + return !candidate.starts_with("http"); + } + + template + bool containerExists(T & blob_service_client, const std::string & container_name) + { + Azure::Storage::Blobs::ListBlobContainersOptions options; + options.Prefix = container_name; + options.PageSizeHint = 1; + + auto containers_list_response = blob_service_client.ListBlobContainers(options); + auto containers_list = containers_list_response.BlobContainers; + + auto it = std::find_if( + containers_list.begin(), containers_list.end(), + [&](const auto & c) { return c.Name == container_name; }); + return it != containers_list.end(); + } +} + +Poco::URI StorageAzureConfiguration::getConnectionURL() const +{ + if (!is_connection_string) + return Poco::URI(connection_url); + + auto parsed_connection_string = Azure::Storage::_internal::ParseConnectionString(connection_url); + return Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); +} + +void StorageAzureConfiguration::check(ContextPtr context) const +{ + context->getGlobalContext()->getRemoteHostFilter().checkURL(getConnectionURL()); + Configuration::check(context); +} + +StorageAzureConfiguration::StorageAzureConfiguration(const StorageAzureConfiguration & other) + : Configuration(other) +{ + connection_url = other.connection_url; + is_connection_string = other.is_connection_string; + account_name = other.account_name; + account_key = other.account_key; + container = other.container; + blob_path = other.blob_path; + blobs_paths = other.blobs_paths; +} + +AzureObjectStorage::SettingsPtr StorageAzureConfiguration::createSettings(ContextPtr context) +{ + const auto & context_settings = context->getSettingsRef(); + auto settings_ptr = std::make_unique(); + settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; + settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; + settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); + settings_ptr->strict_upload_part_size = context_settings.azure_strict_upload_part_size; + settings_ptr->max_upload_part_size = context_settings.azure_max_upload_part_size; + settings_ptr->max_blocks_in_multipart_upload = context_settings.azure_max_blocks_in_multipart_upload; + settings_ptr->min_upload_part_size = context_settings.azure_min_upload_part_size; + return settings_ptr; +} + +StorageObjectStorage::QuerySettings StorageAzureConfiguration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.azure_truncate_on_insert, + .create_new_file_on_insert = settings.azure_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_azure, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.azure_skip_empty_files, + .list_object_keys_size = settings.azure_list_object_keys_size, + .throw_on_zero_files_match = settings.azure_throw_on_zero_files_match, + .ignore_non_existent_file = settings.azure_ignore_file_doesnt_exist, + }; +} + +ObjectStoragePtr StorageAzureConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT +{ + assertInitialized(); + auto client = createClient(is_readonly, /* attempt_to_create_container */true); + auto settings = createSettings(context); + return std::make_unique( + "AzureBlobStorage", std::move(client), std::move(settings), container, getConnectionURL().toString()); +} + +AzureClientPtr StorageAzureConfiguration::createClient(bool is_read_only, bool attempt_to_create_container) +{ + using namespace Azure::Storage::Blobs; + + AzureClientPtr result; + + if (is_connection_string) + { + auto managed_identity_credential = std::make_shared(); + auto blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(connection_url)); + result = std::make_unique(BlobContainerClient::CreateFromConnectionString(connection_url, container)); + + if (attempt_to_create_container) + { + bool container_exists = containerExists(*blob_service_client, container); + if (!container_exists) + { + if (is_read_only) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "AzureBlobStorage container does not exist '{}'", + container); + + try + { + result->CreateIfNotExists(); + } + catch (const Azure::Storage::StorageException & e) + { + if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict + && e.ReasonPhrase == "The specified container already exists.")) + { + throw; + } + } + } + } + } + else + { + std::shared_ptr storage_shared_key_credential; + if (account_name.has_value() && account_key.has_value()) + { + storage_shared_key_credential + = std::make_shared(*account_name, *account_key); + } + + std::unique_ptr blob_service_client; + std::shared_ptr managed_identity_credential; + if (storage_shared_key_credential) + { + blob_service_client = std::make_unique(connection_url, storage_shared_key_credential); + } + else + { + managed_identity_credential = std::make_shared(); + blob_service_client = std::make_unique(connection_url, managed_identity_credential); + } + + std::string final_url; + size_t pos = connection_url.find('?'); + if (pos != std::string::npos) + { + auto url_without_sas = connection_url.substr(0, pos); + final_url = url_without_sas + (url_without_sas.back() == '/' ? "" : "/") + container + + connection_url.substr(pos); + } + else + final_url + = connection_url + (connection_url.back() == '/' ? "" : "/") + container; + + if (!attempt_to_create_container) + { + if (storage_shared_key_credential) + return std::make_unique(final_url, storage_shared_key_credential); + else + return std::make_unique(final_url, managed_identity_credential); + } + + bool container_exists = containerExists(*blob_service_client, container); + if (container_exists) + { + if (storage_shared_key_credential) + result = std::make_unique(final_url, storage_shared_key_credential); + else + result = std::make_unique(final_url, managed_identity_credential); + } + else + { + if (is_read_only) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "AzureBlobStorage container does not exist '{}'", + container); + try + { + result = std::make_unique(blob_service_client->CreateBlobContainer(container).Value); + } catch (const Azure::Storage::StorageException & e) + { + if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict + && e.ReasonPhrase == "The specified container already exists.") + { + if (storage_shared_key_credential) + result = std::make_unique(final_url, storage_shared_key_credential); + else + result = std::make_unique(final_url, managed_identity_credential); + } + else + { + throw; + } + } + } + } + + return result; +} + +void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & collection) +{ + validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); + + if (collection.has("connection_string")) + { + connection_url = collection.get("connection_string"); + is_connection_string = true; + } + + if (collection.has("storage_account_url")) + { + connection_url = collection.get("storage_account_url"); + is_connection_string = false; + } + + container = collection.get("container"); + blob_path = collection.get("blob_path"); + + if (collection.has("account_name")) + account_name = collection.get("account_name"); + + if (collection.has("account_key")) + account_key = collection.get("account_key"); + + structure = collection.getOrDefault("structure", "auto"); + format = collection.getOrDefault("format", format); + compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + + blobs_paths = {blob_path}; +} + +void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure) +{ + if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7)) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage AzureBlobStorage requires 3 to 7 arguments: " + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure)])"); + } + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); + + std::unordered_map engine_args_to_idx; + + connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); + is_connection_string = isConnectionString(connection_url); + + container = checkAndGetLiteralArgument(engine_args[1], "container"); + blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); + + auto is_format_arg = [] (const std::string & s) -> bool + { + return s == "auto" || FormatFactory::instance().getAllFormats().contains(Poco::toLower(s)); + }; + + if (engine_args.size() == 4) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + format = fourth_arg; + } + else + { + if (with_structure) + structure = fourth_arg; + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unknown format or account name specified without account key: {}", fourth_arg); + } + } + else if (engine_args.size() == 5) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + format = fourth_arg; + compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + } + else + { + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + } + } + else if (engine_args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + if (with_structure) + { + format = fourth_arg; + compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + structure = checkAndGetLiteralArgument(engine_args[5], "structure"); + } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); + } + else + { + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (is_format_arg(sixth_arg)) + format = sixth_arg; + else + { + if (with_structure) + structure = sixth_arg; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + } + } + } + else if (engine_args.size() == 7) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (!with_structure && is_format_arg(fourth_arg)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); + } + else + { + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + format = sixth_arg; + compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + } + } + else if (with_structure && engine_args.size() == 8) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + format = sixth_arg; + compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + structure = checkAndGetLiteralArgument(engine_args[7], "structure"); + } + + blobs_paths = {blob_path}; +} + +void StorageAzureConfiguration::addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context) +{ + if (tryGetNamedCollectionWithOverrides(args, context)) + { + /// In case of named collection, just add key-value pair "structure='...'" + /// at the end of arguments to override existed structure. + ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; + auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); + args.push_back(equal_func); + } + else + { + if (args.size() < 3 || args.size() > 8) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Azure requires 3 to 7 arguments: " + "StorageObjectStorage(connection_string|storage_account_url, container_name, " + "blobpath, [account_name, account_key, format, compression, structure])"); + } + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + auto structure_literal = std::make_shared(structure_); + auto format_literal = std::make_shared(format_); + auto is_format_arg + = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); }; + + /// (connection_string, container_name, blobpath) + if (args.size() == 3) + { + args.push_back(format_literal); + /// Add compression = "auto" before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// (connection_string, container_name, blobpath, structure) or + /// (connection_string, container_name, blobpath, format) + /// We can distinguish them by looking at the 4-th argument: check if it's format name or not. + else if (args.size() == 4) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name/structure"); + /// (..., format) -> (..., format, compression, structure) + if (is_format_arg(fourth_arg)) + { + if (fourth_arg == "auto") + args[3] = format_literal; + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// (..., structure) -> (..., format, compression, structure) + else + { + auto structure_arg = args.back(); + args[3] = format_literal; + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + if (fourth_arg == "auto") + args.push_back(structure_literal); + else + args.push_back(structure_arg); + } + } + /// (connection_string, container_name, blobpath, format, compression) or + /// (storage_account_url, container_name, blobpath, account_name, account_key) + /// We can distinguish them by looking at the 4-th argument: check if it's format name or not. + else if (args.size() == 5) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); + /// (..., format, compression) -> (..., format, compression, structure) + if (is_format_arg(fourth_arg)) + { + if (fourth_arg == "auto") + args[3] = format_literal; + args.push_back(structure_literal); + } + /// (..., account_name, account_key) -> (..., account_name, account_key, format, compression, structure) + else + { + args.push_back(format_literal); + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + } + /// (connection_string, container_name, blobpath, format, compression, structure) or + /// (storage_account_url, container_name, blobpath, account_name, account_key, structure) or + /// (storage_account_url, container_name, blobpath, account_name, account_key, format) + else if (args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); + auto sixth_arg = checkAndGetLiteralArgument(args[5], "format/structure"); + + /// (..., format, compression, structure) + if (is_format_arg(fourth_arg)) + { + if (fourth_arg == "auto") + args[3] = format_literal; + if (checkAndGetLiteralArgument(args[5], "structure") == "auto") + args[5] = structure_literal; + } + /// (..., account_name, account_key, format) -> (..., account_name, account_key, format, compression, structure) + else if (is_format_arg(sixth_arg)) + { + if (sixth_arg == "auto") + args[5] = format_literal; + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// (..., account_name, account_key, structure) -> (..., account_name, account_key, format, compression, structure) + else + { + auto structure_arg = args.back(); + args[5] = format_literal; + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + if (sixth_arg == "auto") + args.push_back(structure_literal); + else + args.push_back(structure_arg); + } + } + /// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression) + else if (args.size() == 7) + { + /// (..., format, compression) -> (..., format, compression, structure) + if (checkAndGetLiteralArgument(args[5], "format") == "auto") + args[5] = format_literal; + args.push_back(structure_literal); + } + /// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) + else if (args.size() == 8) + { + if (checkAndGetLiteralArgument(args[5], "format") == "auto") + args[5] = format_literal; + if (checkAndGetLiteralArgument(args[7], "structure") == "auto") + args[7] = structure_literal; + } + } +} + +} + +#endif diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h new file mode 100644 index 00000000000..35b19079ca9 --- /dev/null +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -0,0 +1,77 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE +#include +#include +#include + +namespace DB +{ +class BackupFactory; + +class StorageAzureConfiguration : public StorageObjectStorage::Configuration +{ + friend class BackupReaderAzureBlobStorage; + friend class BackupWriterAzureBlobStorage; + friend void registerBackupEngineAzureBlobStorage(BackupFactory & factory); + +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + static constexpr auto type_name = "azure"; + static constexpr auto engine_name = "Azure"; + + StorageAzureConfiguration() = default; + StorageAzureConfiguration(const StorageAzureConfiguration & other); + + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return engine_name; } + + Path getPath() const override { return blob_path; } + void setPath(const Path & path) override { blob_path = path; } + + const Paths & getPaths() const override { return blobs_paths; } + void setPaths(const Paths & paths) override { blobs_paths = paths; } + + String getNamespace() const override { return container; } + String getDataSourceDescription() const override { return std::filesystem::path(connection_url) / container; } + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + + void check(ContextPtr context) const override; + ConfigurationPtr clone() override { return std::make_shared(*this); } + + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; + + void addStructureAndFormatToArgs( + ASTs & args, + const String & structure_, + const String & format_, + ContextPtr context) override; + +protected: + void fromNamedCollection(const NamedCollection & collection) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + + using AzureClient = Azure::Storage::Blobs::BlobContainerClient; + using AzureClientPtr = std::unique_ptr; + + std::string connection_url; + bool is_connection_string; + + std::optional account_name; + std::optional account_key; + + std::string container; + std::string blob_path; + std::vector blobs_paths; + + AzureClientPtr createClient(bool is_read_only, bool attempt_to_create_container); + AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); + Poco::URI getConnectionURL() const; +}; + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Common.cpp b/src/Storages/ObjectStorage/DataLakes/Common.cpp new file mode 100644 index 00000000000..4830cc52a90 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Common.cpp @@ -0,0 +1,28 @@ +#include "Common.h" +#include +#include +#include + +namespace DB +{ + +std::vector listFiles( + const IObjectStorage & object_storage, + const StorageObjectStorage::Configuration & configuration, + const String & prefix, const String & suffix) +{ + auto key = std::filesystem::path(configuration.getPath()) / prefix; + RelativePathsWithMetadata files_with_metadata; + object_storage.listObjects(key, files_with_metadata, 0); + Strings res; + for (const auto & file_with_metadata : files_with_metadata) + { + const auto & filename = file_with_metadata->relative_path; + if (filename.ends_with(suffix)) + res.push_back(filename); + } + LOG_TRACE(getLogger("DataLakeCommon"), "Listed {} files ({})", res.size(), fmt::join(res, ", ")); + return res; +} + +} diff --git a/src/Storages/ObjectStorage/DataLakes/Common.h b/src/Storages/ObjectStorage/DataLakes/Common.h new file mode 100644 index 00000000000..db3afa9e4a6 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Common.h @@ -0,0 +1,15 @@ +#pragma once +#include +#include + +namespace DB +{ + +class IObjectStorage; + +std::vector listFiles( + const IObjectStorage & object_storage, + const StorageObjectStorage::Configuration & configuration, + const String & prefix, const String & suffix); + +} diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp similarity index 78% rename from src/Storages/DataLakes/DeltaLakeMetadataParser.cpp rename to src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 1687a4754f5..277d07d88ef 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -1,11 +1,9 @@ -#include +#include #include #include "config.h" #include #if USE_AWS_S3 && USE_PARQUET -#include -#include #include #include #include @@ -13,13 +11,12 @@ #include #include #include +#include +#include #include #include #include -#include -#include - -namespace fs = std::filesystem; +#include namespace DB { @@ -30,13 +27,24 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -template -struct DeltaLakeMetadataParser::Impl +struct DeltaLakeMetadata::Impl { + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + ContextPtr context; + /** * Useful links: * - https://github.com/delta-io/delta/blob/master/PROTOCOL.md#data-files */ + Impl(ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_) + : object_storage(object_storage_) + , configuration(configuration_) + , context(context_) + { + } /** * DeltaLake tables store metadata files and data files. @@ -66,10 +74,10 @@ struct DeltaLakeMetadataParser::Impl * An action changes one aspect of the table's state, for example, adding or removing a file. * Note: it is not a valid json, but a list of json's, so we read it in a while cycle. */ - std::set processMetadataFiles(const Configuration & configuration, ContextPtr context) + std::set processMetadataFiles() { std::set result_files; - const auto checkpoint_version = getCheckpointIfExists(result_files, configuration, context); + const auto checkpoint_version = getCheckpointIfExists(result_files); if (checkpoint_version) { @@ -77,12 +85,12 @@ struct DeltaLakeMetadataParser::Impl while (true) { const auto filename = withPadding(++current_version) + metadata_file_suffix; - const auto file_path = fs::path(configuration.getPath()) / deltalake_metadata_directory / filename; + const auto file_path = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / filename; - if (!MetadataReadHelper::exists(file_path, configuration)) + if (!object_storage->exists(StoredObject(file_path))) break; - processMetadataFile(file_path, result_files, configuration, context); + processMetadataFile(file_path, result_files); } LOG_TRACE( @@ -91,11 +99,9 @@ struct DeltaLakeMetadataParser::Impl } else { - const auto keys = MetadataReadHelper::listFiles( - configuration, deltalake_metadata_directory, metadata_file_suffix); - + const auto keys = listFiles(*object_storage, *configuration, deltalake_metadata_directory, metadata_file_suffix); for (const String & key : keys) - processMetadataFile(key, result_files, configuration, context); + processMetadataFile(key, result_files); } return result_files; @@ -130,13 +136,10 @@ struct DeltaLakeMetadataParser::Impl * \"nullCount\":{\"col-6c990940-59bb-4709-8f2e-17083a82c01a\":0,\"col-763cd7e2-7627-4d8e-9fb7-9e85d0c8845b\":0}}"}} * " */ - void processMetadataFile( - const String & key, - std::set & result, - const Configuration & configuration, - ContextPtr context) + void processMetadataFile(const String & key, std::set & result) const { - auto buf = MetadataReadHelper::createReadBuffer(key, context, configuration); + auto read_settings = context->getReadSettings(); + auto buf = object_storage->readObject(StoredObject(key), read_settings); char c; while (!buf->eof()) @@ -158,12 +161,12 @@ struct DeltaLakeMetadataParser::Impl if (json.has("add")) { const auto path = json["add"]["path"].getString(); - result.insert(fs::path(configuration.getPath()) / path); + result.insert(std::filesystem::path(configuration->getPath()) / path); } else if (json.has("remove")) { const auto path = json["remove"]["path"].getString(); - result.erase(fs::path(configuration.getPath()) / path); + result.erase(std::filesystem::path(configuration->getPath()) / path); } } } @@ -181,14 +184,15 @@ struct DeltaLakeMetadataParser::Impl * * We need to get "version", which is the version of the checkpoint we need to read. */ - size_t readLastCheckpointIfExists(const Configuration & configuration, ContextPtr context) + size_t readLastCheckpointIfExists() const { - const auto last_checkpoint_file = fs::path(configuration.getPath()) / deltalake_metadata_directory / "_last_checkpoint"; - if (!MetadataReadHelper::exists(last_checkpoint_file, configuration)) + const auto last_checkpoint_file = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / "_last_checkpoint"; + if (!object_storage->exists(StoredObject(last_checkpoint_file))) return 0; String json_str; - auto buf = MetadataReadHelper::createReadBuffer(last_checkpoint_file, context, configuration); + auto read_settings = context->getReadSettings(); + auto buf = object_storage->readObject(StoredObject(last_checkpoint_file), read_settings); readJSONObjectPossiblyInvalid(json_str, *buf); const JSON json(json_str); @@ -238,18 +242,19 @@ struct DeltaLakeMetadataParser::Impl throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arrow error: {}", _s.ToString()); \ } while (false) - size_t getCheckpointIfExists(std::set & result, const Configuration & configuration, ContextPtr context) + size_t getCheckpointIfExists(std::set & result) { - const auto version = readLastCheckpointIfExists(configuration, context); + const auto version = readLastCheckpointIfExists(); if (!version) return 0; const auto checkpoint_filename = withPadding(version) + ".checkpoint.parquet"; - const auto checkpoint_path = fs::path(configuration.getPath()) / deltalake_metadata_directory / checkpoint_filename; + const auto checkpoint_path = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / checkpoint_filename; LOG_TRACE(log, "Using checkpoint file: {}", checkpoint_path.string()); - auto buf = MetadataReadHelper::createReadBuffer(checkpoint_path, context, configuration); + auto read_settings = context->getReadSettings(); + auto buf = object_storage->readObject(StoredObject(checkpoint_path), read_settings); auto format_settings = getFormatSettings(context); /// Force nullable, because this parquet file for some reason does not have nullable @@ -306,7 +311,7 @@ struct DeltaLakeMetadataParser::Impl if (filename.empty()) continue; LOG_TEST(log, "Adding {}", filename); - const auto [_, inserted] = result.insert(fs::path(configuration.getPath()) / filename); + const auto [_, inserted] = result.insert(std::filesystem::path(configuration->getPath()) / filename); if (!inserted) throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", filename); } @@ -317,22 +322,24 @@ struct DeltaLakeMetadataParser::Impl LoggerPtr log = getLogger("DeltaLakeMetadataParser"); }; - -template -DeltaLakeMetadataParser::DeltaLakeMetadataParser() : impl(std::make_unique()) +DeltaLakeMetadata::DeltaLakeMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_) + : impl(std::make_unique(object_storage_, configuration_, context_)) { } -template -Strings DeltaLakeMetadataParser::getFiles(const Configuration & configuration, ContextPtr context) +Strings DeltaLakeMetadata::getDataFiles() const { - auto result = impl->processMetadataFiles(configuration, context); - return Strings(result.begin(), result.end()); + if (!data_files.empty()) + return data_files; + + auto result = impl->processMetadataFiles(); + data_files = Strings(result.begin(), result.end()); + return data_files; } -template DeltaLakeMetadataParser::DeltaLakeMetadataParser(); -template Strings DeltaLakeMetadataParser::getFiles( - const StorageS3::Configuration & configuration, ContextPtr); } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h new file mode 100644 index 00000000000..e527721b29e --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class DeltaLakeMetadata final : public IDataLakeMetadata +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + static constexpr auto name = "DeltaLake"; + + DeltaLakeMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_); + + Strings getDataFiles() const override; + + NamesAndTypesList getTableSchema() const override { return {}; } + + bool operator ==(const IDataLakeMetadata & other) const override + { + const auto * deltalake_metadata = dynamic_cast(&other); + return deltalake_metadata + && !data_files.empty() && !deltalake_metadata->data_files.empty() + && data_files == deltalake_metadata->data_files; + } + + static DataLakeMetadataPtr create( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + ContextPtr local_context) + { + return std::make_unique(object_storage, configuration, local_context); + } + +private: + struct Impl; + const std::shared_ptr impl; + mutable Strings data_files; +}; + +} diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp new file mode 100644 index 00000000000..91a586ccbf9 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp @@ -0,0 +1,106 @@ +#include +#include +#include +#include +#include +#include +#include "config.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/** + * Useful links: + * - https://hudi.apache.org/tech-specs/ + * - https://hudi.apache.org/docs/file_layouts/ + */ + +/** + * Hudi tables store metadata files and data files. + * Metadata files are stored in .hoodie/metadata directory. Though unlike DeltaLake and Iceberg, + * metadata is not required in order to understand which files we need to read, moreover, + * for Hudi metadata does not always exist. + * + * There can be two types of data files + * 1. base files (columnar file formats like Apache Parquet/Orc) + * 2. log files + * Currently we support reading only `base files`. + * Data file name format: + * [File Id]_[File Write Token]_[Transaction timestamp].[File Extension] + * + * To find needed parts we need to find out latest part file for every file group for every partition. + * Explanation why: + * Hudi reads in and overwrites the entire table/partition with each update. + * Hudi controls the number of file groups under a single partition according to the + * hoodie.parquet.max.file.size option. Once a single Parquet file is too large, Hudi creates a second file group. + * Each file group is identified by File Id. + */ +Strings HudiMetadata::getDataFilesImpl() const +{ + auto log = getLogger("HudiMetadata"); + const auto keys = listFiles(*object_storage, *configuration, "", Poco::toLower(configuration->format)); + + using Partition = std::string; + using FileID = std::string; + struct FileInfo + { + String key; + UInt64 timestamp = 0; + }; + std::unordered_map> files; + + for (const auto & key : keys) + { + auto key_file = std::filesystem::path(key); + Strings file_parts; + const String stem = key_file.stem(); + splitInto<'_'>(file_parts, stem); + if (file_parts.size() != 3) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format for file: {}", key); + + const auto partition = key_file.parent_path().stem(); + const auto & file_id = file_parts[0]; + const auto timestamp = parse(file_parts[2]); + + auto & file_info = files[partition][file_id]; + if (file_info.timestamp == 0 || file_info.timestamp < timestamp) + { + file_info.key = key; + file_info.timestamp = timestamp; + } + } + + Strings result; + for (auto & [partition, partition_data] : files) + { + LOG_TRACE(log, "Adding {} data files from partition {}", partition, partition_data.size()); + for (auto & [file_id, file_data] : partition_data) + result.push_back(std::move(file_data.key)); + } + return result; +} + +HudiMetadata::HudiMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_) + : WithContext(context_) + , object_storage(object_storage_) + , configuration(configuration_) +{ +} + +Strings HudiMetadata::getDataFiles() const +{ + if (data_files.empty()) + data_files = getDataFilesImpl(); + return data_files; +} + +} diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h new file mode 100644 index 00000000000..3ab274b1fbf --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class HudiMetadata final : public IDataLakeMetadata, private WithContext +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + static constexpr auto name = "Hudi"; + + HudiMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_); + + Strings getDataFiles() const override; + + NamesAndTypesList getTableSchema() const override { return {}; } + + bool operator ==(const IDataLakeMetadata & other) const override + { + const auto * hudi_metadata = dynamic_cast(&other); + return hudi_metadata + && !data_files.empty() && !hudi_metadata->data_files.empty() + && data_files == hudi_metadata->data_files; + } + + static DataLakeMetadataPtr create( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + ContextPtr local_context) + { + return std::make_unique(object_storage, configuration, local_context); + } + +private: + const ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; + mutable Strings data_files; + + Strings getDataFilesImpl() const; +}; + +} diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h new file mode 100644 index 00000000000..a2bd5adb947 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h @@ -0,0 +1,19 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class IDataLakeMetadata : boost::noncopyable +{ +public: + virtual ~IDataLakeMetadata() = default; + virtual Strings getDataFiles() const = 0; + virtual NamesAndTypesList getTableSchema() const = 0; + virtual bool operator==(const IDataLakeMetadata & other) const = 0; +}; +using DataLakeMetadataPtr = std::unique_ptr; + +} diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h new file mode 100644 index 00000000000..83865c47eb8 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -0,0 +1,139 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 && USE_AVRO + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Storage for read-only integration with Apache Iceberg tables in Amazon S3 (see https://iceberg.apache.org/) +/// Right now it's implemented on top of StorageS3 and right now it doesn't support +/// many Iceberg features like schema evolution, partitioning, positional and equality deletes. +template +class IStorageDataLake final : public StorageObjectStorage +{ +public: + using Storage = StorageObjectStorage; + using ConfigurationPtr = Storage::ConfigurationPtr; + + static StoragePtr create( + ConfigurationPtr base_configuration, + ContextPtr context, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment_, + std::optional format_settings_, + LoadingStrictnessLevel mode) + { + auto object_storage = base_configuration->createObjectStorage(context, /* is_readonly */true); + DataLakeMetadataPtr metadata; + NamesAndTypesList schema_from_metadata; + + if (base_configuration->format == "auto") + base_configuration->format = "Parquet"; + + ConfigurationPtr configuration = base_configuration->clone(); + + try + { + metadata = DataLakeMetadata::create(object_storage, base_configuration, context); + schema_from_metadata = metadata->getTableSchema(); + configuration->setPaths(metadata->getDataFiles()); + } + catch (...) + { + if (mode <= LoadingStrictnessLevel::CREATE) + throw; + + metadata.reset(); + configuration->setPaths({}); + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + return std::make_shared>( + base_configuration, std::move(metadata), configuration, object_storage, + context, table_id_, + columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, + constraints_, comment_, format_settings_); + } + + String getName() const override { return DataLakeMetadata::name; } + + static ColumnsDescription getTableStructureFromData( + ObjectStoragePtr object_storage_, + ConfigurationPtr base_configuration, + const std::optional & format_settings_, + ContextPtr local_context) + { + auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); + + auto schema_from_metadata = metadata->getTableSchema(); + if (schema_from_metadata != NamesAndTypesList{}) + { + return ColumnsDescription(std::move(schema_from_metadata)); + } + else + { + ConfigurationPtr configuration = base_configuration->clone(); + configuration->setPaths(metadata->getDataFiles()); + return Storage::resolveSchemaFromData( + object_storage_, configuration, format_settings_, local_context); + } + } + + void updateConfiguration(ContextPtr local_context) override + { + Storage::updateConfiguration(local_context); + + auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); + + if (current_metadata && *current_metadata == *new_metadata) + return; + + current_metadata = std::move(new_metadata); + auto updated_configuration = base_configuration->clone(); + updated_configuration->setPaths(current_metadata->getDataFiles()); + + Storage::configuration = updated_configuration; + } + + template + IStorageDataLake( + ConfigurationPtr base_configuration_, + DataLakeMetadataPtr metadata_, + Args &&... args) + : Storage(std::forward(args)...) + , base_configuration(base_configuration_) + , current_metadata(std::move(metadata_)) + { + if (base_configuration->format == "auto") + { + base_configuration->format = Storage::configuration->format; + } + } + +private: + ConfigurationPtr base_configuration; + DataLakeMetadataPtr current_metadata; +}; + +using StorageIceberg = IStorageDataLake; +using StorageDeltaLake = IStorageDataLake; +using StorageHudi = IStorageDataLake; + +} + +#endif diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp similarity index 93% rename from src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp rename to src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index a50fc2972df..0484f86542c 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -21,11 +21,11 @@ #include #include #include +#include #include #include -#include -#include -#include +#include +#include #include #include @@ -45,7 +45,8 @@ extern const int UNSUPPORTED_METHOD; } IcebergMetadata::IcebergMetadata( - const StorageS3::Configuration & configuration_, + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, DB::ContextPtr context_, Int32 metadata_version_, Int32 format_version_, @@ -53,6 +54,7 @@ IcebergMetadata::IcebergMetadata( Int32 current_schema_id_, DB::NamesAndTypesList schema_) : WithContext(context_) + , object_storage(object_storage_) , configuration(configuration_) , metadata_version(metadata_version_) , format_version(format_version_) @@ -338,15 +340,17 @@ MutableColumns parseAvro( * 1) v.metadata.json, where V - metadata version. * 2) -.metadata.json, where V - metadata version */ -std::pair getMetadataFileAndVersion(const StorageS3::Configuration & configuration) +std::pair getMetadataFileAndVersion( + ObjectStoragePtr object_storage, + const StorageObjectStorage::Configuration & configuration) { - const auto metadata_files = S3DataLakeMetadataReadHelper::listFiles(configuration, "metadata", ".metadata.json"); + const auto metadata_files = listFiles(*object_storage, configuration, "metadata", ".metadata.json"); if (metadata_files.empty()) { throw Exception( ErrorCodes::FILE_DOESNT_EXIST, "The metadata file for Iceberg table with path {} doesn't exist", - configuration.url.key); + configuration.getPath()); } std::vector> metadata_files_with_versions; @@ -373,11 +377,15 @@ std::pair getMetadataFileAndVersion(const StorageS3::Configuratio } -std::unique_ptr parseIcebergMetadata(const StorageS3::Configuration & configuration, ContextPtr context_) +DataLakeMetadataPtr IcebergMetadata::create( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + ContextPtr local_context) { - const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(configuration); + const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(object_storage, *configuration); LOG_DEBUG(getLogger("IcebergMetadata"), "Parse metadata {}", metadata_file_path); - auto buf = S3DataLakeMetadataReadHelper::createReadBuffer(metadata_file_path, context_, configuration); + auto read_settings = local_context->getReadSettings(); + auto buf = object_storage->readObject(StoredObject(metadata_file_path), read_settings); String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); @@ -386,7 +394,7 @@ std::unique_ptr parseIcebergMetadata(const StorageS3::Configura Poco::JSON::Object::Ptr object = json.extract(); auto format_version = object->getValue("format-version"); - auto [schema, schema_id] = parseTableSchema(object, format_version, context_->getSettingsRef().iceberg_engine_ignore_schema_evolution); + auto [schema, schema_id] = parseTableSchema(object, format_version, local_context->getSettingsRef().iceberg_engine_ignore_schema_evolution); auto current_snapshot_id = object->getValue("current-snapshot-id"); auto snapshots = object->get("snapshots").extract(); @@ -398,12 +406,12 @@ std::unique_ptr parseIcebergMetadata(const StorageS3::Configura if (snapshot->getValue("snapshot-id") == current_snapshot_id) { const auto path = snapshot->getValue("manifest-list"); - manifest_list_file = std::filesystem::path(configuration.url.key) / "metadata" / std::filesystem::path(path).filename(); + manifest_list_file = std::filesystem::path(configuration->getPath()) / "metadata" / std::filesystem::path(path).filename(); break; } } - return std::make_unique(configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); + return std::make_unique(object_storage, configuration, local_context, metadata_version, format_version, manifest_list_file, schema_id, schema); } /** @@ -431,7 +439,7 @@ std::unique_ptr parseIcebergMetadata(const StorageS3::Configura * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=2/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00003.parquet','PARQUET',(2),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'3')],[(1,'\0\0\0\0\0\0\0'),(2,'3')],NULL,[4],0) │ * └────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ */ -Strings IcebergMetadata::getDataFiles() +Strings IcebergMetadata::getDataFiles() const { if (!data_files.empty()) return data_files; @@ -442,12 +450,14 @@ Strings IcebergMetadata::getDataFiles() LOG_TEST(log, "Collect manifest files from manifest list {}", manifest_list_file); - auto manifest_list_buf = S3DataLakeMetadataReadHelper::createReadBuffer(manifest_list_file, getContext(), configuration); + auto context = getContext(); + auto read_settings = context->getReadSettings(); + auto manifest_list_buf = object_storage->readObject(StoredObject(manifest_list_file), read_settings); auto manifest_list_file_reader = std::make_unique(std::make_unique(*manifest_list_buf)); auto data_type = AvroSchemaReader::avroNodeToDataType(manifest_list_file_reader->dataSchema().root()->leafAt(0)); Block header{{data_type->createColumn(), data_type, "manifest_path"}}; - auto columns = parseAvro(*manifest_list_file_reader, header, getFormatSettings(getContext())); + auto columns = parseAvro(*manifest_list_file_reader, header, getFormatSettings(context)); auto & col = columns.at(0); if (col->getDataType() != TypeIndex::String) @@ -463,7 +473,7 @@ Strings IcebergMetadata::getDataFiles() { const auto file_path = col_str->getDataAt(i).toView(); const auto filename = std::filesystem::path(file_path).filename(); - manifest_files.emplace_back(std::filesystem::path(configuration.url.key) / "metadata" / filename); + manifest_files.emplace_back(std::filesystem::path(configuration->getPath()) / "metadata" / filename); } NameSet files; @@ -472,7 +482,7 @@ Strings IcebergMetadata::getDataFiles() { LOG_TEST(log, "Process manifest file {}", manifest_file); - auto buffer = S3DataLakeMetadataReadHelper::createReadBuffer(manifest_file, getContext(), configuration); + auto buffer = object_storage->readObject(StoredObject(manifest_file), read_settings); auto manifest_file_reader = std::make_unique(std::make_unique(*buffer)); /// Manifest file should always have table schema in avro file metadata. By now we don't support tables with evolved schema, @@ -483,7 +493,7 @@ Strings IcebergMetadata::getDataFiles() Poco::JSON::Parser parser; Poco::Dynamic::Var json = parser.parse(schema_json_string); Poco::JSON::Object::Ptr schema_object = json.extract(); - if (!getContext()->getSettingsRef().iceberg_engine_ignore_schema_evolution && schema_object->getValue("schema-id") != current_schema_id) + if (!context->getSettingsRef().iceberg_engine_ignore_schema_evolution && schema_object->getValue("schema-id") != current_schema_id) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " @@ -596,9 +606,9 @@ Strings IcebergMetadata::getDataFiles() const auto status = status_int_column->getInt(i); const auto data_path = std::string(file_path_string_column->getDataAt(i).toView()); - const auto pos = data_path.find(configuration.url.key); + const auto pos = data_path.find(configuration->getPath()); if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration.url.key, data_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration->getPath(), data_path); const auto file_path = data_path.substr(pos); diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h similarity index 69% rename from src/Storages/DataLakes/Iceberg/IcebergMetadata.h rename to src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index 3e6a2ec3415..06dbd373bf9 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -2,9 +2,11 @@ #if USE_AWS_S3 && USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. -#include #include #include +#include +#include +#include namespace DB { @@ -56,40 +58,55 @@ namespace DB * "metadata-log" : [ ] * } */ -class IcebergMetadata : WithContext +class IcebergMetadata : public IDataLakeMetadata, private WithContext { public: - IcebergMetadata(const StorageS3::Configuration & configuration_, - ContextPtr context_, - Int32 metadata_version_, - Int32 format_version_, - String manifest_list_file_, - Int32 current_schema_id_, - NamesAndTypesList schema_); + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + static constexpr auto name = "Iceberg"; + + IcebergMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_, + Int32 metadata_version_, + Int32 format_version_, + String manifest_list_file_, + Int32 current_schema_id_, + NamesAndTypesList schema_); /// Get data files. On first request it reads manifest_list file and iterates through manifest files to find all data files. /// All subsequent calls will return saved list of files (because it cannot be changed without changing metadata file) - Strings getDataFiles(); + Strings getDataFiles() const override; /// Get table schema parsed from metadata. - NamesAndTypesList getTableSchema() const { return schema; } + NamesAndTypesList getTableSchema() const override { return schema; } - size_t getVersion() const { return metadata_version; } + bool operator ==(const IDataLakeMetadata & other) const override + { + const auto * iceberg_metadata = dynamic_cast(&other); + return iceberg_metadata && getVersion() == iceberg_metadata->getVersion(); + } + + static DataLakeMetadataPtr create( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + ContextPtr local_context); private: - const StorageS3::Configuration configuration; + size_t getVersion() const { return metadata_version; } + + const ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; Int32 metadata_version; Int32 format_version; String manifest_list_file; Int32 current_schema_id; NamesAndTypesList schema; - Strings data_files; + mutable Strings data_files; LoggerPtr log; - }; -std::unique_ptr parseIcebergMetadata(const StorageS3::Configuration & configuration, ContextPtr context); - } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp new file mode 100644 index 00000000000..0fa6402e892 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -0,0 +1,82 @@ +#include "config.h" + +#if USE_AWS_S3 + +#include +#include +#include +#include + + +namespace DB +{ + +#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. + +void registerStorageIceberg(StorageFactory & factory) +{ + factory.registerStorage( + "Iceberg", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, + args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +#endif + +#if USE_PARQUET +void registerStorageDeltaLake(StorageFactory & factory) +{ + factory.registerStorage( + "DeltaLake", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageDeltaLake::create( + configuration, args.getContext(), args.table_id, args.columns, + args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} +#endif + +void registerStorageHudi(StorageFactory & factory) +{ + factory.registerStorage( + "Hudi", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageHudi::create( + configuration, args.getContext(), args.table_id, args.columns, + args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +} + +#endif diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp similarity index 99% rename from src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp rename to src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp index 6b6151f5474..21df7e35284 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -1,9 +1,9 @@ #include "AsynchronousReadBufferFromHDFS.h" #if USE_HDFS +#include "ReadBufferFromHDFS.h" #include #include -#include #include #include diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h similarity index 96% rename from src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h rename to src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h index 10e2749fd4a..5aef92315a4 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp new file mode 100644 index 00000000000..a8a9ab5b557 --- /dev/null +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -0,0 +1,217 @@ +#include + +#if USE_HDFS +#include +#include +#include +#include + +#include +#include + +#include +#include +#include + +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other) + : Configuration(other) +{ + url = other.url; + path = other.path; + paths = other.paths; +} + +void StorageHDFSConfiguration::check(ContextPtr context) const +{ + context->getRemoteHostFilter().checkURL(Poco::URI(url)); + checkHDFSURL(fs::path(url) / path.substr(1)); + Configuration::check(context); +} + +ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT + ContextPtr context, + bool /* is_readonly */) +{ + assertInitialized(); + const auto & settings = context->getSettingsRef(); + auto hdfs_settings = std::make_unique( + settings.remote_read_min_bytes_for_seek, + settings.hdfs_replication + ); + return std::make_shared( + url, std::move(hdfs_settings), context->getConfigRef(), /* lazy_initialize */true); +} + +std::string StorageHDFSConfiguration::getPathWithoutGlobs() const +{ + /// Unlike s3 and azure, which are object storages, + /// hdfs is a filesystem, so it cannot list files by partual prefix, + /// only by directory. + auto first_glob_pos = path.find_first_of("*?{"); + auto end_of_path_without_globs = path.substr(0, first_glob_pos).rfind('/'); + if (end_of_path_without_globs == std::string::npos || end_of_path_without_globs == 0) + return "/"; + return path.substr(0, end_of_path_without_globs); +} +StorageObjectStorage::QuerySettings StorageHDFSConfiguration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.hdfs_truncate_on_insert, + .create_new_file_on_insert = settings.hdfs_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_hdfs, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.hdfs_skip_empty_files, + .list_object_keys_size = 0, /// HDFS does not support listing in batches. + .throw_on_zero_files_match = settings.hdfs_throw_on_zero_files_match, + .ignore_non_existent_file = settings.hdfs_ignore_file_doesnt_exist, + }; +} + +void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure) +{ + const size_t max_args_num = with_structure ? 4 : 3; + if (args.empty() || args.size() > max_args_num) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Expected not more than {} arguments", max_args_num); + } + + std::string url_str; + url_str = checkAndGetLiteralArgument(args[0], "url"); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + if (args.size() > 1) + { + format = checkAndGetLiteralArgument(args[1], "format_name"); + } + + if (with_structure) + { + if (args.size() > 2) + { + structure = checkAndGetLiteralArgument(args[2], "structure"); + } + if (args.size() > 3) + { + compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); + } + } + else if (args.size() > 2) + { + compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); + } + + setURL(url_str); +} + +void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection & collection) +{ + std::string url_str; + + auto filename = collection.getOrDefault("filename", ""); + if (!filename.empty()) + url_str = std::filesystem::path(collection.get("url")) / filename; + else + url_str = collection.get("url"); + + format = collection.getOrDefault("format", "auto"); + compression_method = collection.getOrDefault("compression_method", + collection.getOrDefault("compression", "auto")); + structure = collection.getOrDefault("structure", "auto"); + + setURL(url_str); +} + +void StorageHDFSConfiguration::setURL(const std::string & url_) +{ + auto pos = url_.find("//"); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); + + pos = url_.find('/', pos + 2); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); + + path = url_.substr(pos + 1); + if (!path.starts_with('/')) + path = '/' + path; + + url = url_.substr(0, pos); + paths = {path}; + + LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using url: {}, path: {}", url, path); +} + +void StorageHDFSConfiguration::addStructureAndFormatToArgs( + ASTs & args, + const String & structure_, + const String & format_, + ContextPtr context) +{ + if (tryGetNamedCollectionWithOverrides(args, context)) + { + /// In case of named collection, just add key-value pair "structure='...'" + /// at the end of arguments to override existed structure. + ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; + auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); + args.push_back(equal_func); + } + else + { + size_t count = args.size(); + if (count == 0 || count > 4) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Expected 1 to 4 arguments in table function, got {}", count); + } + + auto format_literal = std::make_shared(format_); + auto structure_literal = std::make_shared(structure_); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + /// hdfs(url) + if (count == 1) + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// hdfs(url, format) + else if (count == 2) + { + if (checkAndGetLiteralArgument(args[1], "format") == "auto") + args.back() = format_literal; + args.push_back(structure_literal); + } + /// hdfs(url, format, structure) + /// hdfs(url, format, structure, compression_method) + else if (count >= 3) + { + if (checkAndGetLiteralArgument(args[1], "format") == "auto") + args[1] = format_literal; + if (checkAndGetLiteralArgument(args[2], "structure") == "auto") + args[2] = structure_literal; + } + } +} + +} + +#endif diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h new file mode 100644 index 00000000000..01a8b9c5e3b --- /dev/null +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -0,0 +1,60 @@ +#pragma once +#include "config.h" + +#if USE_HDFS +#include +#include +#include + +namespace DB +{ + +class StorageHDFSConfiguration : public StorageObjectStorage::Configuration +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + static constexpr auto type_name = "hdfs"; + static constexpr auto engine_name = "HDFS"; + + StorageHDFSConfiguration() = default; + StorageHDFSConfiguration(const StorageHDFSConfiguration & other); + + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return engine_name; } + + Path getPath() const override { return path; } + void setPath(const Path & path_) override { path = path_; } + + const Paths & getPaths() const override { return paths; } + void setPaths(const Paths & paths_) override { paths = paths_; } + std::string getPathWithoutGlobs() const override; + + String getNamespace() const override { return ""; } + String getDataSourceDescription() const override { return url; } + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + + void check(ContextPtr context) const override; + ConfigurationPtr clone() override { return std::make_shared(*this); } + + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; + + void addStructureAndFormatToArgs( + ASTs & args, + const String & structure_, + const String & format_, + ContextPtr context) override; + +private: + void fromNamedCollection(const NamedCollection &) override; + void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; + void setURL(const std::string & url_); + + String url; + String path; + std::vector paths; +}; + +} + +#endif diff --git a/src/Storages/HDFS/HDFSCommon.cpp b/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp similarity index 99% rename from src/Storages/HDFS/HDFSCommon.cpp rename to src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp index 9eb0d10cc16..365828bcc40 100644 --- a/src/Storages/HDFS/HDFSCommon.cpp +++ b/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp @@ -1,4 +1,4 @@ -#include +#include "HDFSCommon.h" #include #include #include diff --git a/src/Storages/HDFS/HDFSCommon.h b/src/Storages/ObjectStorage/HDFS/HDFSCommon.h similarity index 100% rename from src/Storages/HDFS/HDFSCommon.h rename to src/Storages/ObjectStorage/HDFS/HDFSCommon.h diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp similarity index 98% rename from src/Storages/HDFS/ReadBufferFromHDFS.cpp rename to src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index 4df05d47003..be339d021dc 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -1,11 +1,12 @@ #include "ReadBufferFromHDFS.h" #if USE_HDFS -#include +#include "HDFSCommon.h" #include #include #include #include +#include #include #include @@ -55,10 +56,10 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(use_external_buffer_ ? 0 : read_settings_.remote_fs_buffer_size) , hdfs_uri(hdfs_uri_) , hdfs_file_path(hdfs_file_path_) - , builder(createHDFSBuilder(hdfs_uri_, config_)) , read_settings(read_settings_) , read_until_position(read_until_position_) { + builder = createHDFSBuilder(hdfs_uri_, config_); fs = createHDFSFS(builder.get()); fin = hdfsOpenFile(fs.get(), hdfs_file_path.c_str(), O_RDONLY, 0, 0, 0); @@ -100,7 +101,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory {})", file_offset, read_until_position - 1); @@ -145,6 +148,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemoryadd(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); + return true; } diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h similarity index 100% rename from src/Storages/HDFS/ReadBufferFromHDFS.h rename to src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp similarity index 93% rename from src/Storages/HDFS/WriteBufferFromHDFS.cpp rename to src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 173dd899ada..2c14b38ce01 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -2,8 +2,8 @@ #if USE_HDFS -#include -#include +#include "WriteBufferFromHDFS.h" +#include "HDFSCommon.h" #include #include #include @@ -48,12 +48,13 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); const String path = hdfs_uri.substr(begin_of_path); - fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here + /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here + fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); if (fout == nullptr) { - throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Unable to open HDFS file: {} error: {}", - path, std::string(hdfsGetLastError())); + throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Unable to open HDFS file: {} ({}) error: {}", + path, hdfs_uri, std::string(hdfsGetLastError())); } } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h similarity index 100% rename from src/Storages/HDFS/WriteBufferFromHDFS.h rename to src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp new file mode 100644 index 00000000000..5e89a0a1b9d --- /dev/null +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -0,0 +1,293 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int CANNOT_DETECT_FORMAT; +} + +ReadBufferIterator::ReadBufferIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const FileIterator & file_iterator_, + const std::optional & format_settings_, + SchemaCache & schema_cache_, + ObjectInfos & read_keys_, + const ContextPtr & context_) + : WithContext(context_) + , object_storage(object_storage_) + , configuration(configuration_) + , file_iterator(file_iterator_) + , format_settings(format_settings_) + , query_settings(configuration->getQuerySettings(context_)) + , schema_cache(schema_cache_) + , read_keys(read_keys_) + , prev_read_keys_size(read_keys_.size()) +{ + if (configuration->format != "auto") + format = configuration->format; +} + +SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const ObjectInfo & object_info, const String & format_name) const +{ + auto source = StorageObjectStorageSource::getUniqueStoragePathIdentifier(*configuration, object_info); + return DB::getKeyForSchemaCache(source, format_name, format_settings, getContext()); +} + +SchemaCache::Keys ReadBufferIterator::getKeysForSchemaCache() const +{ + Strings sources; + sources.reserve(read_keys.size()); + std::transform( + read_keys.begin(), read_keys.end(), + std::back_inserter(sources), + [&](const auto & elem) + { + return StorageObjectStorageSource::getUniqueStoragePathIdentifier(*configuration, *elem); + }); + return DB::getKeysForSchemaCache(sources, *format, format_settings, getContext()); +} + +std::optional ReadBufferIterator::tryGetColumnsFromCache( + const ObjectInfos::iterator & begin, + const ObjectInfos::iterator & end) +{ + if (!query_settings.schema_inference_use_cache) + return std::nullopt; + + for (auto it = begin; it < end; ++it) + { + const auto & object_info = (*it); + auto get_last_mod_time = [&] -> std::optional + { + const auto & path = object_info->isArchive() ? object_info->getPathToArchive() : object_info->getPath(); + if (!object_info->metadata) + object_info->metadata = object_storage->tryGetObjectMetadata(path); + + return object_info->metadata + ? std::optional(object_info->metadata->last_modified.epochTime()) + : std::nullopt; + }; + + if (format) + { + const auto cache_key = getKeyForSchemaCache(*object_info, *format); + if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) + return columns; + } + else + { + /// If format is unknown, we can iterate through all possible input formats + /// and check if we have an entry with this format and this file in schema cache. + /// If we have such entry for some format, we can use this format to read the file. + for (const auto & format_name : FormatFactory::instance().getAllInputFormats()) + { + const auto cache_key = getKeyForSchemaCache(*object_info, format_name); + if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) + { + /// Now format is known. It should be the same for all files. + format = format_name; + return columns; + } + } + } + } + return std::nullopt; +} + +void ReadBufferIterator::setNumRowsToLastFile(size_t num_rows) +{ + if (query_settings.schema_inference_use_cache) + schema_cache.addNumRows(getKeyForSchemaCache(*current_object_info, *format), num_rows); +} + +void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) +{ + if (query_settings.schema_inference_use_cache + && query_settings.schema_inference_mode == SchemaInferenceMode::UNION) + { + schema_cache.addColumns(getKeyForSchemaCache(*current_object_info, *format), columns); + } +} + +void ReadBufferIterator::setResultingSchema(const ColumnsDescription & columns) +{ + if (query_settings.schema_inference_use_cache + && query_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + schema_cache.addManyColumns(getKeysForSchemaCache(), columns); + } +} + +void ReadBufferIterator::setFormatName(const String & format_name) +{ + format = format_name; +} + +String ReadBufferIterator::getLastFileName() const +{ + if (current_object_info) + return current_object_info->getPath(); + else + return ""; +} + +std::unique_ptr ReadBufferIterator::recreateLastReadBuffer() +{ + auto context = getContext(); + + const auto & path = current_object_info->isArchive() ? current_object_info->getPathToArchive() : current_object_info->getPath(); + auto impl = object_storage->readObject(StoredObject(path), context->getReadSettings()); + + const auto compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); + const auto zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); + + return wrapReadBufferWithCompressionMethod(std::move(impl), compression_method, zstd_window_log_max); +} + +ReadBufferIterator::Data ReadBufferIterator::next() +{ + if (first) + { + /// If format is unknown we iterate through all currently read keys on first iteration and + /// try to determine format by file name. + if (!format) + { + for (const auto & object_info : read_keys) + { + if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(object_info->getFileName())) + { + format = format_from_file_name; + break; + } + } + } + + /// For default mode check cached columns for currently read keys on first iteration. + if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) + { + return {nullptr, cached_columns, format}; + } + } + } + + while (true) + { + current_object_info = file_iterator->next(0); + + if (!current_object_info) + { + if (first) + { + if (format.has_value()) + { + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "The table structure cannot be extracted from a {} format file, " + "because there are no files with provided path " + "in {} or all files are empty. You can specify table structure manually", + *format, object_storage->getName()); + } + + throw Exception( + ErrorCodes::CANNOT_DETECT_FORMAT, + "The data format cannot be detected by the contents of the files, " + "because there are no files with provided path " + "in {} or all files are empty. You can specify the format manually", + object_storage->getName()); + } + + return {nullptr, std::nullopt, format}; + } + + const auto filename = current_object_info->getFileName(); + chassert(!filename.empty()); + + /// file iterator could get new keys after new iteration + if (read_keys.size() > prev_read_keys_size) + { + /// If format is unknown we can try to determine it by new file names. + if (!format) + { + for (auto it = read_keys.begin() + prev_read_keys_size; it != read_keys.end(); ++it) + { + if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->getFileName())) + { + format = format_from_file_name; + break; + } + } + } + + /// Check new files in schema cache if schema inference mode is default. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); + if (columns_from_cache) + return {nullptr, columns_from_cache, format}; + } + + prev_read_keys_size = read_keys.size(); + } + + if (query_settings.skip_empty_files + && current_object_info->metadata && current_object_info->metadata->size_bytes == 0) + continue; + + /// In union mode, check cached columns only for current key. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + ObjectInfos objects{current_object_info}; + if (auto columns_from_cache = tryGetColumnsFromCache(objects.begin(), objects.end())) + { + first = false; + return {nullptr, columns_from_cache, format}; + } + } + + LOG_TEST(getLogger("KSSENII"), "Will read columns from {}", current_object_info->getPath()); + + std::unique_ptr read_buf; + CompressionMethod compression_method; + using ObjectInfoInArchive = StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive; + if (const auto * object_info_in_archive = dynamic_cast(current_object_info.get())) + { + LOG_TEST(getLogger("KSSENII"), "Will read columns from {} from archive", current_object_info->getPath()); + compression_method = chooseCompressionMethod(filename, configuration->compression_method); + const auto & archive_reader = object_info_in_archive->archive_reader; + read_buf = archive_reader->readFile(object_info_in_archive->path_in_archive, /*throw_on_not_found=*/true); + } + else + { + LOG_TEST(getLogger("KSSENII"), "Will read columns from {} from s3", current_object_info->getPath()); + compression_method = chooseCompressionMethod(filename, configuration->compression_method); + read_buf = object_storage->readObject( + StoredObject(current_object_info->getPath()), + getContext()->getReadSettings(), + {}, + current_object_info->metadata->size_bytes); + } + + if (!query_settings.skip_empty_files || !read_buf->eof()) + { + first = false; + + read_buf = wrapReadBufferWithCompressionMethod( + std::move(read_buf), + compression_method, + static_cast(getContext()->getSettingsRef().zstd_window_log_max)); + + return {std::move(read_buf), std::nullopt, format}; + } + } +} +} diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h new file mode 100644 index 00000000000..6eeb52ec2ed --- /dev/null +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -0,0 +1,63 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ + +class ReadBufferIterator : public IReadBufferIterator, WithContext +{ +public: + using FileIterator = std::shared_ptr; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + using ObjectInfoPtr = StorageObjectStorage::ObjectInfoPtr; + using ObjectInfo = StorageObjectStorage::ObjectInfo; + using ObjectInfos = StorageObjectStorage::ObjectInfos; + + ReadBufferIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const FileIterator & file_iterator_, + const std::optional & format_settings_, + SchemaCache & schema_cache_, + ObjectInfos & read_keys_, + const ContextPtr & context_); + + Data next() override; + + void setNumRowsToLastFile(size_t num_rows) override; + + void setSchemaToLastFile(const ColumnsDescription & columns) override; + + void setResultingSchema(const ColumnsDescription & columns) override; + + String getLastFileName() const override; + + void setFormatName(const String & format_name) override; + + bool supportsLastReadBufferRecreation() const override { return true; } + + std::unique_ptr recreateLastReadBuffer() override; + +private: + SchemaCache::Key getKeyForSchemaCache(const ObjectInfo & object_info, const String & format_name) const; + SchemaCache::Keys getKeysForSchemaCache() const; + std::optional tryGetColumnsFromCache( + const ObjectInfos::iterator & begin, const ObjectInfos::iterator & end); + + ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; + const FileIterator file_iterator; + const std::optional & format_settings; + const StorageObjectStorage::QuerySettings query_settings; + SchemaCache & schema_cache; + ObjectInfos & read_keys; + std::optional format; + + size_t prev_read_keys_size; + ObjectInfoPtr current_object_info; + bool first = true; +}; +} diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp new file mode 100644 index 00000000000..4b217b94730 --- /dev/null +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -0,0 +1,472 @@ +#include + +#if USE_AWS_S3 +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include + +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; +} + +static const std::unordered_set required_configuration_keys = { + "url", +}; + +static const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "compression_method", + "structure", + "access_key_id", + "secret_access_key", + "session_token", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", + "expiration_window_seconds", + "no_sign_request" +}; + +String StorageS3Configuration::getDataSourceDescription() const +{ + return std::filesystem::path(url.uri.getHost() + std::to_string(url.uri.getPort())) / url.bucket; +} + +std::string StorageS3Configuration::getPathInArchive() const +{ + if (url.archive_pattern.has_value()) + return url.archive_pattern.value(); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Path {} is not an archive", getPath()); +} + +void StorageS3Configuration::check(ContextPtr context) const +{ + validateNamespace(url.bucket); + context->getGlobalContext()->getRemoteHostFilter().checkURL(url.uri); + context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(headers_from_ast); + Configuration::check(context); +} + +void StorageS3Configuration::validateNamespace(const String & name) const +{ + S3::URI::validateBucket(name, {}); +} + +StorageS3Configuration::StorageS3Configuration(const StorageS3Configuration & other) + : Configuration(other) +{ + url = other.url; + static_configuration = other.static_configuration; + headers_from_ast = other.headers_from_ast; + keys = other.keys; +} + +StorageObjectStorage::QuerySettings StorageS3Configuration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.s3_truncate_on_insert, + .create_new_file_on_insert = settings.s3_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_s3, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.s3_skip_empty_files, + .list_object_keys_size = settings.s3_list_object_keys_size, + .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .ignore_non_existent_file = settings.s3_ignore_file_doesnt_exist, + }; +} + +ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT +{ + assertInitialized(); + + const auto & config = context->getConfigRef(); + const auto & settings = context->getSettingsRef(); + const std::string config_prefix = "s3."; + + auto s3_settings = getSettings(config, config_prefix, context, settings.s3_validate_request_settings); + + request_settings.updateFromSettingsIfChanged(settings); + auth_settings.updateFrom(s3_settings->auth_settings); + + s3_settings->auth_settings = auth_settings; + s3_settings->request_settings = request_settings; + + if (!headers_from_ast.empty()) + { + s3_settings->auth_settings.headers.insert( + s3_settings->auth_settings.headers.end(), + headers_from_ast.begin(), headers_from_ast.end()); + } + + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) + s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); + + auto client = getClient(config, config_prefix, context, *s3_settings, false, &url); + auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); + auto s3_capabilities = S3Capabilities + { + .support_batch_delete = config.getBool("s3.support_batch_delete", true), + .support_proxy = config.getBool("s3.support_proxy", config.has("s3.proxy")), + }; + + return std::make_shared( + std::move(client), std::move(s3_settings), url, s3_capabilities, + key_generator, "StorageS3", false); +} + +void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection) +{ + validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); + + auto filename = collection.getOrDefault("filename", ""); + if (!filename.empty()) + url = S3::URI(std::filesystem::path(collection.get("url")) / filename); + else + url = S3::URI(collection.get("url")); + + auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); + auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); + auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 1); + auth_settings.no_sign_request = collection.getOrDefault("no_sign_request", false); + auth_settings.expiration_window_seconds = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); + + format = collection.getOrDefault("format", format); + compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + structure = collection.getOrDefault("structure", "auto"); + + request_settings = S3Settings::RequestSettings(collection); + + static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); + + keys = {url.key}; +} + +void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_structure) +{ + /// Supported signatures: S3('url') S3('url', 'format') S3('url', 'format', 'compression') S3('url', NOSIGN) S3('url', NOSIGN, 'format') S3('url', NOSIGN, 'format', 'compression') S3('url', 'aws_access_key_id', 'aws_secret_access_key') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format', 'compression') + /// with optional headers() function + + size_t count = StorageURL::evalArgsAndCollectHeaders(args, headers_from_ast, context); + + if (count == 0 || count > (with_structure ? 7 : 6)) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage S3 requires 1 to 5 arguments: " + "url, [NOSIGN | access_key_id, secret_access_key], name of used format and [compression_method]"); + + std::unordered_map engine_args_to_idx; + bool no_sign_request = false; + + /// For 2 arguments we support 2 possible variants: + /// - s3(source, format) + /// - s3(source, NOSIGN) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. + if (count == 2) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + no_sign_request = true; + else + engine_args_to_idx = {{"format", 1}}; + } + /// For 3 arguments we support 2 possible variants: + /// - s3(source, format, compression_method) + /// - s3(source, access_key_id, secret_access_key) + /// - s3(source, NOSIGN, format) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name. + else if (count == 3) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + engine_args_to_idx = {{"format", 2}}; + } + else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) + { + if (with_structure) + engine_args_to_idx = {{"format", 1}, {"structure", 2}}; + else + engine_args_to_idx = {{"format", 1}, {"compression_method", 2}}; + } + else + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; + } + /// For 4 arguments we support 3 possible variants: + /// if with_structure == 0: + /// - s3(source, access_key_id, secret_access_key, session_token) + /// - s3(source, access_key_id, secret_access_key, format) + /// - s3(source, NOSIGN, format, compression_method) + /// if with_structure == 1: + /// - s3(source, format, structure, compression_method), + /// - s3(source, access_key_id, secret_access_key, format), + /// - s3(source, access_key_id, secret_access_key, session_token) + /// - s3(source, NOSIGN, format, structure) + /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not. + else if (count == 4) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "access_key_id/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + if (with_structure) + engine_args_to_idx = {{"format", 2}, {"structure", 3}}; + else + engine_args_to_idx = {{"format", 2}, {"compression_method", 3}}; + } + else if (with_structure && (second_arg == "auto" || FormatFactory::instance().exists(second_arg))) + { + engine_args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; + } + else + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "session_token/format"); + if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}}; + } + } + } + /// For 5 arguments we support 2 possible variants: + /// if with_structure == 0: + /// - s3(source, access_key_id, secret_access_key, session_token, format) + /// - s3(source, access_key_id, secret_access_key, format, compression) + /// if with_structure == 1: + /// - s3(source, access_key_id, secret_access_key, format, structure) + /// - s3(source, access_key_id, secret_access_key, session_token, format) + /// - s3(source, NOSIGN, format, structure, compression_method) + else if (count == 5) + { + if (with_structure) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "NOSIGN/access_key_id"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + engine_args_to_idx = {{"format", 2}, {"structure", 3}, {"compression_method", 4}}; + } + else + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); + if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}; + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}}; + } + } + } + else + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "session_token/format"); + if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"compression_method", 4}}; + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}}; + } + } + } + else if (count == 6) + { + if (with_structure) + { + /// - s3(source, access_key_id, secret_access_key, format, structure, compression_method) + /// - s3(source, access_key_id, secret_access_key, session_token, format, structure) + /// We can distinguish them by looking at the 4-th argument: check if it's a format name or not + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); + if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}; + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}}; + } + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"compression_method", 5}}; + } + } + else if (with_structure && count == 7) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}; + } + + /// This argument is always the first + url = S3::URI(checkAndGetLiteralArgument(args[0], "url")); + + if (engine_args_to_idx.contains("format")) + { + format = checkAndGetLiteralArgument(args[engine_args_to_idx["format"]], "format"); + /// Set format to configuration only of it's not 'auto', + /// because we can have default format set in configuration. + if (format != "auto") + format = format; + } + + if (engine_args_to_idx.contains("structure")) + structure = checkAndGetLiteralArgument(args[engine_args_to_idx["structure"]], "structure"); + + if (engine_args_to_idx.contains("compression_method")) + compression_method = checkAndGetLiteralArgument(args[engine_args_to_idx["compression_method"]], "compression_method"); + + if (engine_args_to_idx.contains("access_key_id")) + auth_settings.access_key_id = checkAndGetLiteralArgument(args[engine_args_to_idx["access_key_id"]], "access_key_id"); + + if (engine_args_to_idx.contains("secret_access_key")) + auth_settings.secret_access_key = checkAndGetLiteralArgument(args[engine_args_to_idx["secret_access_key"]], "secret_access_key"); + + if (engine_args_to_idx.contains("session_token")) + auth_settings.session_token = checkAndGetLiteralArgument(args[engine_args_to_idx["session_token"]], "session_token"); + + if (no_sign_request) + auth_settings.no_sign_request = no_sign_request; + + static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); + auth_settings.no_sign_request = no_sign_request; + + keys = {url.key}; +} + +void StorageS3Configuration::addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context) +{ + if (tryGetNamedCollectionWithOverrides(args, context)) + { + /// In case of named collection, just add key-value pair "structure='...'" + /// at the end of arguments to override existed structure. + ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; + auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); + args.push_back(equal_func); + } + else + { + HTTPHeaderEntries tmp_headers; + size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context); + + if (count == 0 || count > 6) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to 6 arguments in table function, got {}", count); + + auto format_literal = std::make_shared(format_); + auto structure_literal = std::make_shared(structure_); + + /// s3(s3_url) + if (count == 1) + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// s3(s3_url, format) or s3(s3_url, NOSIGN) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. + else if (count == 2) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + /// If there is NOSIGN, add format=auto before structure. + if (boost::iequals(second_arg, "NOSIGN")) + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// s3(source, format, structure) or + /// s3(source, access_key_id, secret_access_key) or + /// s3(source, NOSIGN, format) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. + else if (count == 3) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + args.push_back(structure_literal); + } + else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) + { + args[count - 1] = structure_literal; + } + else + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + } + /// s3(source, format, structure, compression_method) or + /// s3(source, access_key_id, secret_access_key, format) or + /// s3(source, NOSIGN, format, structure) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. + else if (count == 4) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + args[count - 1] = structure_literal; + } + else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) + { + args[count - 2] = structure_literal; + } + else + { + args.push_back(structure_literal); + } + } + /// s3(source, access_key_id, secret_access_key, format, structure) or + /// s3(source, NOSIGN, format, structure, compression_method) + /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not. + else if (count == 5) + { + auto sedond_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(sedond_arg, "NOSIGN")) + { + args[count - 2] = structure_literal; + } + else + { + args[count - 1] = structure_literal; + } + } + /// s3(source, access_key_id, secret_access_key, format, structure, compression) + else if (count == 6) + { + args[count - 2] = structure_literal; + } + } +} + +} + +#endif diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h new file mode 100644 index 00000000000..906d10a1a9a --- /dev/null +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -0,0 +1,70 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 +#include +#include + +namespace DB +{ + +class StorageS3Configuration : public StorageObjectStorage::Configuration +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + static constexpr auto type_name = "s3"; + static constexpr auto namespace_name = "bucket"; + + StorageS3Configuration() = default; + StorageS3Configuration(const StorageS3Configuration & other); + + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return url.storage_name; } + std::string getNamespaceType() const override { return namespace_name; } + + Path getPath() const override { return url.key; } + void setPath(const Path & path) override { url.key = path; } + + const Paths & getPaths() const override { return keys; } + void setPaths(const Paths & paths) override { keys = paths; } + + String getNamespace() const override { return url.bucket; } + String getDataSourceDescription() const override; + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + + bool isArchive() const override { return url.archive_pattern.has_value(); } + std::string getPathInArchive() const override; + + void check(ContextPtr context) const override; + void validateNamespace(const String & name) const override; + ConfigurationPtr clone() override { return std::make_shared(*this); } + bool isStaticConfiguration() const override { return static_configuration; } + + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; + + void addStructureAndFormatToArgs( + ASTs & args, + const String & structure, + const String & format, + ContextPtr context) override; + +private: + void fromNamedCollection(const NamedCollection & collection) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + + S3::URI url; + std::vector keys; + + S3::AuthSettings auth_settings; + S3Settings::RequestSettings request_settings; + HTTPHeaderEntries headers_from_ast; /// Headers from ast is a part of static configuration. + /// If s3 configuration was passed from ast, then it is static. + /// If from config - it can be changed with config reload. + bool static_configuration = true; +}; + +} + +#endif diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp new file mode 100644 index 00000000000..2c8e60b49d0 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -0,0 +1,503 @@ +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DATABASE_ACCESS_DENIED; + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; +} + +StorageObjectStorage::StorageObjectStorage( + ConfigurationPtr configuration_, + ObjectStoragePtr object_storage_, + ContextPtr context, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_, + bool distributed_processing_, + ASTPtr partition_by_) + : IStorage(table_id_) + , configuration(configuration_) + , object_storage(object_storage_) + , format_settings(format_settings_) + , partition_by(partition_by_) + , distributed_processing(distributed_processing_) + , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) +{ + ColumnsDescription columns{columns_}; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context); + configuration->check(context); + + StorageInMemoryMetadata metadata; + metadata.setColumns(columns); + metadata.setConstraints(constraints_); + metadata.setComment(comment); + + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); + setInMemoryMetadata(metadata); +} + +String StorageObjectStorage::getName() const +{ + return configuration->getEngineName(); +} + +bool StorageObjectStorage::prefersLargeBlocks() const +{ + return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration->format); +} + +bool StorageObjectStorage::parallelizeOutputAfterReading(ContextPtr context) const +{ + return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration->format, context); +} + +bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) const +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context, format_settings); +} + +void StorageObjectStorage::updateConfiguration(ContextPtr context) +{ + IObjectStorage::ApplyNewSettingsOptions options{ .allow_client_change = !configuration->isStaticConfiguration() }; + object_storage->applyNewSettings(context->getConfigRef(), configuration->getTypeName() + ".", context, options); +} + +namespace +{ +class ReadFromObjectStorageStep : public SourceStepWithFilter +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + ReadFromObjectStorageStep( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const String & name_, + const Names & columns_to_read, + const NamesAndTypesList & virtual_columns_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const std::optional & format_settings_, + bool distributed_processing_, + ReadFromFormatInfo info_, + const bool need_only_count_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = info_.source_header}, columns_to_read, query_info_, storage_snapshot_, context_) + , object_storage(object_storage_) + , configuration(configuration_) + , info(std::move(info_)) + , virtual_columns(virtual_columns_) + , format_settings(format_settings_) + , name(name_ + "Source") + , need_only_count(need_only_count_) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + , distributed_processing(distributed_processing_) + { + } + + std::string getName() const override { return name; } + + void applyFilters(ActionDAGNodes added_filter_nodes) override + { + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + createIterator(predicate); + } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + createIterator(nullptr); + + Pipes pipes; + auto context = getContext(); + const size_t max_threads = context->getSettingsRef().max_threads; + size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); + + if (estimated_keys_count > 1) + num_streams = std::min(num_streams, estimated_keys_count); + else + { + /// The amount of keys (zero) was probably underestimated. + /// We will keep one stream for this particular case. + num_streams = 1; + } + + const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared( + getName(), object_storage, configuration, info, format_settings, + context, max_block_size, iterator_wrapper, max_parsing_threads, need_only_count); + + source->setKeyCondition(filter_actions_dag, context); + pipes.emplace_back(std::move(source)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); + } + +private: + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + std::shared_ptr iterator_wrapper; + + const ReadFromFormatInfo info; + const NamesAndTypesList virtual_columns; + const std::optional format_settings; + const String name; + const bool need_only_count; + const size_t max_block_size; + size_t num_streams; + const bool distributed_processing; + + void createIterator(const ActionsDAG::Node * predicate) + { + if (iterator_wrapper) + return; + auto context = getContext(); + iterator_wrapper = StorageObjectStorageSource::createFileIterator( + configuration, object_storage, distributed_processing, + context, predicate, virtual_columns, nullptr, context->getFileProgressCallback()); + } +}; +} + +void StorageObjectStorage::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ + updateConfiguration(local_context); + if (partition_by && configuration->withPartitionWildcard()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Reading from a partitioned {} storage is not implemented yet", + getName()); + } + + const auto read_from_format_info = prepareReadingFromFormat( + column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); + const bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) + && local_context->getSettingsRef().optimize_count_from_files; + + auto read_step = std::make_unique( + object_storage, + configuration, + getName(), + column_names, + getVirtualsList(), + query_info, + storage_snapshot, + format_settings, + distributed_processing, + read_from_format_info, + need_only_count, + local_context, + max_block_size, + num_streams); + + query_plan.addStep(std::move(read_step)); +} + +SinkToStoragePtr StorageObjectStorage::write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context, + bool /* async_insert */) +{ + updateConfiguration(local_context); + const auto sample_block = metadata_snapshot->getSampleBlock(); + const auto & settings = configuration->getQuerySettings(local_context); + + if (configuration->isArchive()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Path '{}' contains archive. Write into archive is not supported", + configuration->getPath()); + } + + if (configuration->withGlobsIgnorePartitionWildcard()) + { + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, + "Path '{}' contains globs, so the table is in readonly mode", + configuration->getPath()); + } + + if (configuration->withPartitionWildcard()) + { + ASTPtr partition_by_ast = nullptr; + if (auto insert_query = std::dynamic_pointer_cast(query)) + { + if (insert_query->partition_by) + partition_by_ast = insert_query->partition_by; + else + partition_by_ast = partition_by; + } + + if (partition_by_ast) + { + return std::make_shared( + object_storage, configuration, format_settings, sample_block, local_context, partition_by_ast); + } + } + + auto paths = configuration->getPaths(); + if (auto new_key = checkAndGetNewFileOnInsertIfNeeded( + *object_storage, *configuration, settings, paths.front(), paths.size())) + { + paths.push_back(*new_key); + } + configuration->setPaths(paths); + + return std::make_shared( + object_storage, + configuration->clone(), + format_settings, + sample_block, + local_context); +} + +void StorageObjectStorage::truncate( + const ASTPtr & /* query */, + const StorageMetadataPtr & /* metadata_snapshot */, + ContextPtr /* context */, + TableExclusiveLockHolder & /* table_holder */) +{ + if (configuration->isArchive()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Path '{}' contains archive. Table cannot be truncated", + configuration->getPath()); + } + + if (configuration->withGlobs()) + { + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "{} key '{}' contains globs, so the table is in readonly mode and cannot be truncated", + getName(), configuration->getPath()); + } + + StoredObjects objects; + for (const auto & key : configuration->getPaths()) + objects.emplace_back(key); + + object_storage->removeObjectsIfExist(objects); +} + +std::unique_ptr StorageObjectStorage::createReadBufferIterator( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + ObjectInfos & read_keys, + const ContextPtr & context) +{ + auto file_iterator = StorageObjectStorageSource::createFileIterator( + configuration, + object_storage, + false/* distributed_processing */, + context, + {}/* predicate */, + {}/* virtual_columns */, + &read_keys); + + return std::make_unique( + object_storage, configuration, file_iterator, + format_settings, getSchemaCache(context, configuration->getTypeName()), read_keys, context); +} + +ColumnsDescription StorageObjectStorage::resolveSchemaFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context) +{ + ObjectInfos read_keys; + auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + return readSchemaFromFormat(configuration->format, format_settings, *iterator, context); +} + +std::string StorageObjectStorage::resolveFormatFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context) +{ + ObjectInfos read_keys; + auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + return detectFormatAndReadSchema(format_settings, *iterator, context).second; +} + +std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context) +{ + ObjectInfos read_keys; + auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); + configuration->format = format; + return std::pair(columns, format); +} + +SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, const std::string & storage_type_name) +{ + if (storage_type_name == "s3") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + "schema_inference_cache_max_elements_for_s3", + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } + else if (storage_type_name == "hdfs") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + "schema_inference_cache_max_elements_for_hdfs", + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } + else if (storage_type_name == "azure") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + "schema_inference_cache_max_elements_for_azure", + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); +} + +void StorageObjectStorage::Configuration::initialize( + Configuration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure) +{ + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + configuration.fromNamedCollection(*named_collection); + else + configuration.fromAST(engine_args, local_context, with_table_structure); + + if (configuration.format == "auto") + { + configuration.format = FormatFactory::instance().tryGetFormatFromFileName( + configuration.isArchive() + ? configuration.getPathInArchive() + : configuration.getPath()).value_or("auto"); + } + else + FormatFactory::instance().checkFormatName(configuration.format); + + configuration.initialized = true; +} + +void StorageObjectStorage::Configuration::check(ContextPtr) const +{ + FormatFactory::instance().checkFormatName(format); +} + +StorageObjectStorage::Configuration::Configuration(const Configuration & other) +{ + format = other.format; + compression_method = other.compression_method; + structure = other.structure; +} + +bool StorageObjectStorage::Configuration::withPartitionWildcard() const +{ + static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + return getPath().find(PARTITION_ID_WILDCARD) != String::npos + || getNamespace().find(PARTITION_ID_WILDCARD) != String::npos; +} + +bool StorageObjectStorage::Configuration::withGlobsIgnorePartitionWildcard() const +{ + if (!withPartitionWildcard()) + return withGlobs(); + else + return PartitionedSink::replaceWildcards(getPath(), "").find_first_of("*?{") != std::string::npos; +} + +bool StorageObjectStorage::Configuration::isPathWithGlobs() const +{ + return getPath().find_first_of("*?{") != std::string::npos; +} + +bool StorageObjectStorage::Configuration::isNamespaceWithGlobs() const +{ + return getNamespace().find_first_of("*?{") != std::string::npos; +} + +std::string StorageObjectStorage::Configuration::getPathWithoutGlobs() const +{ + return getPath().substr(0, getPath().find_first_of("*?{")); +} + +bool StorageObjectStorage::Configuration::isPathInArchiveWithGlobs() const +{ + return getPathInArchive().find_first_of("*?{") != std::string::npos; +} + +std::string StorageObjectStorage::Configuration::getPathInArchive() const +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "Path {} is not archive", getPath()); +} + +void StorageObjectStorage::Configuration::assertInitialized() const +{ + if (!initialized) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration was not initialized before usage"); + } +} +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h new file mode 100644 index 00000000000..f45d8c1f01a --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -0,0 +1,204 @@ +#pragma once +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class ReadBufferIterator; +class SchemaCache; +class NamedCollection; + +/** + * A general class containing implementation for external table engines + * such as StorageS3, StorageAzure, StorageHDFS. + * Works with an object of IObjectStorage class. + */ +class StorageObjectStorage : public IStorage +{ +public: + class Configuration; + using ConfigurationPtr = std::shared_ptr; + using ObjectInfo = RelativePathWithMetadata; + using ObjectInfoPtr = std::shared_ptr; + using ObjectInfos = std::vector; + + struct QuerySettings + { + /// Insert settings: + bool truncate_on_insert; + bool create_new_file_on_insert; + + /// Schema inference settings: + bool schema_inference_use_cache; + SchemaInferenceMode schema_inference_mode; + + /// List settings: + bool skip_empty_files; + size_t list_object_keys_size; + bool throw_on_zero_files_match; + bool ignore_non_existent_file; + }; + + StorageObjectStorage( + ConfigurationPtr configuration_, + ObjectStoragePtr object_storage_, + ContextPtr context_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_, + bool distributed_processing_ = false, + ASTPtr partition_by_ = nullptr); + + String getName() const override; + + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) override; + + void truncate( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context, + TableExclusiveLockHolder &) override; + + bool supportsPartitionBy() const override { return true; } + + bool supportsSubcolumns() const override { return true; } + + bool supportsDynamicSubcolumns() const override { return true; } + + bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } + + bool supportsSubsetOfColumns(const ContextPtr & context) const; + + bool prefersLargeBlocks() const override; + + bool parallelizeOutputAfterReading(ContextPtr context) const override; + + static SchemaCache & getSchemaCache(const ContextPtr & context, const std::string & storage_type_name); + + static ColumnsDescription resolveSchemaFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context); + + static std::string resolveFormatFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context); + + static std::pair resolveSchemaAndFormatFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context); + +protected: + virtual void updateConfiguration(ContextPtr local_context); + + static std::unique_ptr createReadBufferIterator( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + ObjectInfos & read_keys, + const ContextPtr & context); + + ConfigurationPtr configuration; + const ObjectStoragePtr object_storage; + const std::optional format_settings; + const ASTPtr partition_by; + const bool distributed_processing; + + LoggerPtr log; +}; + +class StorageObjectStorage::Configuration +{ +public: + Configuration() = default; + Configuration(const Configuration & other); + virtual ~Configuration() = default; + + using Path = std::string; + using Paths = std::vector; + + static void initialize( + Configuration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure); + + /// Storage type: s3, hdfs, azure. + virtual std::string getTypeName() const = 0; + /// Engine name: S3, HDFS, Azure. + virtual std::string getEngineName() const = 0; + /// Sometimes object storages have something similar to chroot or namespace, for example + /// buckets in S3. If object storage doesn't have any namepaces return empty string. + virtual std::string getNamespaceType() const { return "namespace"; } + + virtual Path getPath() const = 0; + virtual void setPath(const Path & path) = 0; + + virtual const Paths & getPaths() const = 0; + virtual void setPaths(const Paths & paths) = 0; + + virtual String getDataSourceDescription() const = 0; + virtual String getNamespace() const = 0; + + virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; + virtual void addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0; + + bool withPartitionWildcard() const; + bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); } + bool withGlobsIgnorePartitionWildcard() const; + bool isPathWithGlobs() const; + bool isNamespaceWithGlobs() const; + virtual std::string getPathWithoutGlobs() const; + + virtual bool isArchive() const { return false; } + bool isPathInArchiveWithGlobs() const; + virtual std::string getPathInArchive() const; + + virtual void check(ContextPtr context) const; + virtual void validateNamespace(const String & /* name */) const {} + + virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) = 0; + virtual ConfigurationPtr clone() = 0; + virtual bool isStaticConfiguration() const { return true; } + + String format = "auto"; + String compression_method = "auto"; + String structure = "auto"; + +protected: + virtual void fromNamedCollection(const NamedCollection & collection) = 0; + virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; + + void assertInitialized() const; + + bool initialized = false; +}; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp new file mode 100644 index 00000000000..78f568d8ae2 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -0,0 +1,100 @@ +#include "Storages/ObjectStorage/StorageObjectStorageCluster.h" + +#include +#include +#include +#include + +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +StorageObjectStorageCluster::StorageObjectStorageCluster( + const String & cluster_name_, + ConfigurationPtr configuration_, + ObjectStoragePtr object_storage_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ContextPtr context_) + : IStorageCluster( + cluster_name_, table_id_, getLogger(fmt::format("{}({})", configuration_->getEngineName(), table_id_.table_name))) + , configuration{configuration_} + , object_storage(object_storage_) +{ + ColumnsDescription columns{columns_}; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, context_); + configuration->check(context_); + + StorageInMemoryMetadata metadata; + metadata.setColumns(columns); + metadata.setConstraints(constraints_); + + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); + setInMemoryMetadata(metadata); +} + +std::string StorageObjectStorageCluster::getName() const +{ + return configuration->getEngineName(); +} + +void StorageObjectStorageCluster::updateQueryToSendIfNeeded( + ASTPtr & query, + const DB::StorageSnapshotPtr & storage_snapshot, + const ContextPtr & context) +{ + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); + if (!expression_list) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Expected SELECT query from table function {}, got '{}'", + configuration->getEngineName(), queryToString(query)); + } + + ASTs & args = expression_list->children; + const auto & structure = storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(); + if (args.empty()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unexpected empty list of arguments for {}Cluster table function", + configuration->getEngineName()); + } + + ASTPtr cluster_name_arg = args.front(); + args.erase(args.begin()); + configuration->addStructureAndFormatToArgs(args, structure, configuration->format, context); + args.insert(args.begin(), cluster_name_arg); +} + +RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExtension( + const ActionsDAG::Node * predicate, const ContextPtr & local_context) const +{ + auto iterator = StorageObjectStorageSource::createFileIterator( + configuration, object_storage, /* distributed_processing */false, local_context, + predicate, virtual_columns, nullptr, local_context->getFileProgressCallback()); + + auto callback = std::make_shared>([iterator]() mutable -> String + { + auto object_info = iterator->next(0); + if (object_info) + return object_info->getPath(); + else + return ""; + }); + return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; +} + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h new file mode 100644 index 00000000000..69fec2b3c77 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -0,0 +1,48 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class Context; + +class StorageObjectStorageCluster : public IStorageCluster +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + StorageObjectStorageCluster( + const String & cluster_name_, + ConfigurationPtr configuration_, + ObjectStoragePtr object_storage_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ContextPtr context_); + + std::string getName() const override; + + bool supportsSubcolumns() const override { return true; } + + bool supportsDynamicSubcolumns() const override { return true; } + + bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } + + RemoteQueryExecutor::Extension getTaskIteratorExtension( + const ActionsDAG::Node * predicate, const ContextPtr & context) const override; + +private: + void updateQueryToSendIfNeeded( + ASTPtr & query, + const StorageSnapshotPtr & storage_snapshot, + const ContextPtr & context) override; + + const String engine_name; + const StorageObjectStorage::ConfigurationPtr configuration; + const ObjectStoragePtr object_storage; + NamesAndTypesList virtual_columns; +}; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp new file mode 100644 index 00000000000..0a3cf19a590 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -0,0 +1,168 @@ +#include "StorageObjectStorageSink.h" +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; + extern const int BAD_ARGUMENTS; +} + +StorageObjectStorageSink::StorageObjectStorageSink( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + const std::optional & format_settings_, + const Block & sample_block_, + ContextPtr context, + const std::string & blob_path) + : SinkToStorage(sample_block_) + , sample_block(sample_block_) +{ + const auto & settings = context->getSettingsRef(); + const auto path = blob_path.empty() ? configuration->getPaths().back() : blob_path; + const auto chosen_compression_method = chooseCompressionMethod(path, configuration->compression_method); + + auto buffer = object_storage->writeObject( + StoredObject(path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + + write_buf = wrapWriteBufferWithCompressionMethod( + std::move(buffer), + chosen_compression_method, + static_cast(settings.output_format_compression_level), + static_cast(settings.output_format_compression_zstd_window_log)); + + writer = FormatFactory::instance().getOutputFormatParallelIfPossible( + configuration->format, *write_buf, sample_block, context, format_settings_); +} + +void StorageObjectStorageSink::consume(Chunk chunk) +{ + std::lock_guard lock(cancel_mutex); + if (cancelled) + return; + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); +} + +void StorageObjectStorageSink::onCancel() +{ + std::lock_guard lock(cancel_mutex); + finalize(); + cancelled = true; +} + +void StorageObjectStorageSink::onException(std::exception_ptr exception) +{ + std::lock_guard lock(cancel_mutex); + try + { + std::rethrow_exception(exception); + } + catch (...) + { + /// An exception context is needed to proper delete write buffers without finalization. + release(); + } +} + +void StorageObjectStorageSink::onFinish() +{ + std::lock_guard lock(cancel_mutex); + finalize(); +} + +void StorageObjectStorageSink::finalize() +{ + if (!writer) + return; + + try + { + writer->finalize(); + writer->flush(); + write_buf->finalize(); + } + catch (...) + { + /// Stop ParallelFormattingOutputFormat correctly. + release(); + throw; + } +} + +void StorageObjectStorageSink::release() +{ + writer.reset(); + write_buf.reset(); +} + +PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + std::optional format_settings_, + const Block & sample_block_, + ContextPtr context_, + const ASTPtr & partition_by) + : PartitionedSink(partition_by, context_, sample_block_) + , object_storage(object_storage_) + , configuration(configuration_) + , query_settings(configuration_->getQuerySettings(context_)) + , format_settings(format_settings_) + , sample_block(sample_block_) + , context(context_) +{ +} + +SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String & partition_id) +{ + auto partition_bucket = replaceWildcards(configuration->getNamespace(), partition_id); + validateNamespace(partition_bucket); + + auto partition_key = replaceWildcards(configuration->getPath(), partition_id); + validateKey(partition_key); + + if (auto new_key = checkAndGetNewFileOnInsertIfNeeded( + *object_storage, *configuration, query_settings, partition_key, /* sequence_number */1)) + { + partition_key = *new_key; + } + + return std::make_shared( + object_storage, + configuration, + format_settings, + sample_block, + context, + partition_key + ); +} + +void PartitionedStorageObjectStorageSink::validateKey(const String & str) +{ + /// See: + /// - https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-keys.html + /// - https://cloud.ibm.com/apidocs/cos/cos-compatibility#putobject + + if (str.empty() || str.size() > 1024) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incorrect key length (not empty, max 1023 characters), got: {}", str.size()); + + if (!UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in key"); + + validatePartitionKey(str, true); +} + +void PartitionedStorageObjectStorageSink::validateNamespace(const String & str) +{ + configuration->validateNamespace(str); + + if (!UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in bucket name"); + + validatePartitionKey(str, false); +} + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h new file mode 100644 index 00000000000..45cf83d606f --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -0,0 +1,70 @@ +#pragma once +#include +#include +#include + +namespace DB +{ +class StorageObjectStorageSink : public SinkToStorage +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + StorageObjectStorageSink( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + const std::optional & format_settings_, + const Block & sample_block_, + ContextPtr context, + const std::string & blob_path = ""); + + String getName() const override { return "StorageObjectStorageSink"; } + + void consume(Chunk chunk) override; + + void onCancel() override; + + void onException(std::exception_ptr exception) override; + + void onFinish() override; + +private: + const Block sample_block; + std::unique_ptr write_buf; + OutputFormatPtr writer; + bool cancelled = false; + std::mutex cancel_mutex; + + void finalize(); + void release(); +}; + +class PartitionedStorageObjectStorageSink : public PartitionedSink +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + PartitionedStorageObjectStorageSink( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + std::optional format_settings_, + const Block & sample_block_, + ContextPtr context_, + const ASTPtr & partition_by); + + SinkPtr createSinkForPartition(const String & partition_id) override; + +private: + void validateKey(const String & str); + void validateNamespace(const String & str); + + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + + const StorageObjectStorage::QuerySettings query_settings; + const std::optional format_settings; + const Block sample_block; + const ContextPtr context; +}; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp new file mode 100644 index 00000000000..b31d0f8a92e --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -0,0 +1,783 @@ +#include "StorageObjectStorageSource.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace fs = std::filesystem; + +namespace ProfileEvents +{ + extern const Event EngineFileLikeReadFiles; +} + +namespace CurrentMetrics +{ + extern const Metric StorageObjectStorageThreads; + extern const Metric StorageObjectStorageThreadsActive; + extern const Metric StorageObjectStorageThreadsScheduled; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPILE_REGEXP; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int FILE_DOESNT_EXIST; +} + +StorageObjectStorageSource::StorageObjectStorageSource( + String name_, + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const ReadFromFormatInfo & info, + const std::optional & format_settings_, + ContextPtr context_, + UInt64 max_block_size_, + std::shared_ptr file_iterator_, + size_t max_parsing_threads_, + bool need_only_count_) + : SourceWithKeyCondition(info.source_header, false) + , WithContext(context_) + , name(std::move(name_)) + , object_storage(object_storage_) + , configuration(configuration_) + , format_settings(format_settings_) + , max_block_size(max_block_size_) + , need_only_count(need_only_count_) + , max_parsing_threads(max_parsing_threads_) + , read_from_format_info(info) + , create_reader_pool(std::make_shared( + CurrentMetrics::StorageObjectStorageThreads, + CurrentMetrics::StorageObjectStorageThreadsActive, + CurrentMetrics::StorageObjectStorageThreadsScheduled, + 1/* max_threads */)) + , columns_desc(info.columns_description) + , file_iterator(file_iterator_) + , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) + , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) +{ +} + +StorageObjectStorageSource::~StorageObjectStorageSource() +{ + create_reader_pool->wait(); +} + +void StorageObjectStorageSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) +{ + setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); +} + +std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( + const Configuration & configuration, + const ObjectInfo & object_info, + bool include_connection_info) +{ + auto path = object_info.getPath(); + if (path.starts_with("/")) + path = path.substr(1); + + if (include_connection_info) + return fs::path(configuration.getDataSourceDescription()) / path; + else + return fs::path(configuration.getNamespace()) / path; +} + +std::shared_ptr StorageObjectStorageSource::createFileIterator( + ConfigurationPtr configuration, + ObjectStoragePtr object_storage, + bool distributed_processing, + const ContextPtr & local_context, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns, + ObjectInfos * read_keys, + std::function file_progress_callback) +{ + if (distributed_processing) + return std::make_shared( + local_context->getReadTaskCallback(), + local_context->getSettingsRef().max_threads); + + if (configuration->isNamespaceWithGlobs()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expression can not have wildcards inside {} name", configuration->getNamespaceType()); + + auto settings = configuration->getQuerySettings(local_context); + const bool is_archive = configuration->isArchive(); + + std::unique_ptr iterator; + if (configuration->isPathWithGlobs()) + { + /// Iterate through disclosed globs and make a source for each file + iterator = std::make_unique( + object_storage, configuration, predicate, virtual_columns, + local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size, + settings.throw_on_zero_files_match, file_progress_callback); + } + else + { + ConfigurationPtr copy_configuration = configuration->clone(); + auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + if (filter_dag) + { + auto keys = configuration->getPaths(); + std::vector paths; + paths.reserve(keys.size()); + for (const auto & key : keys) + paths.push_back(fs::path(configuration->getNamespace()) / key); + VirtualColumnUtils::filterByPathOrFile(keys, paths, filter_dag, virtual_columns, local_context); + copy_configuration->setPaths(keys); + } + + iterator = std::make_unique( + object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, + settings.ignore_non_existent_file, file_progress_callback); + } + + if (is_archive) + { + return std::make_shared(object_storage, configuration, std::move(iterator), local_context, read_keys); + } + + return iterator; +} + +void StorageObjectStorageSource::lazyInitialize(size_t processor) +{ + if (initialized) + return; + + reader = createReader(processor); + if (reader) + reader_future = createReaderAsync(processor); + initialized = true; +} + +Chunk StorageObjectStorageSource::generate() +{ + lazyInitialize(0); + + while (true) + { + if (isCancelled() || !reader) + { + if (reader) + reader->cancel(); + break; + } + + Chunk chunk; + if (reader->pull(chunk)) + { + UInt64 num_rows = chunk.getNumRows(); + total_rows_in_file += num_rows; + + size_t chunk_size = 0; + if (const auto * input_format = reader.getInputFormat()) + chunk_size = input_format->getApproxBytesReadForChunk(); + + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); + + const auto & object_info = reader.getObjectInfo(); + const auto & filename = object_info.getFileName(); + chassert(object_info.metadata); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( + chunk, + read_from_format_info.requested_virtual_columns, + getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), + object_info.metadata->size_bytes, &filename); + + return chunk; + } + + if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) + addNumRowsToCache(reader.getObjectInfo(), total_rows_in_file); + + total_rows_in_file = 0; + + assert(reader_future.valid()); + reader = reader_future.get(); + + if (!reader) + break; + + /// Even if task is finished the thread may be not freed in pool. + /// So wait until it will be freed before scheduling a new task. + create_reader_pool->wait(); + reader_future = createReaderAsync(); + } + + return {}; +} + +void StorageObjectStorageSource::addNumRowsToCache(const ObjectInfo & object_info, size_t num_rows) +{ + const auto cache_key = getKeyForSchemaCache( + getUniqueStoragePathIdentifier(*configuration, object_info), + configuration->format, + format_settings, + getContext()); + schema_cache.addNumRows(cache_key, num_rows); +} + +std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfo & object_info) +{ + const auto cache_key = getKeyForSchemaCache( + getUniqueStoragePathIdentifier(*configuration, object_info), + configuration->format, + format_settings, + getContext()); + + auto get_last_mod_time = [&]() -> std::optional + { + return object_info.metadata + ? std::optional(object_info.metadata->last_modified.epochTime()) + : std::nullopt; + }; + return schema_cache.tryGetNumRows(cache_key, get_last_mod_time); +} + +StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReader(size_t processor) +{ + ObjectInfoPtr object_info; + auto query_settings = configuration->getQuerySettings(getContext()); + + do + { + object_info = file_iterator->next(processor); + + if (!object_info || object_info->getFileName().empty()) + return {}; + + if (!object_info->metadata) + { + const auto & path = object_info->isArchive() ? object_info->getPathToArchive() : object_info->getPath(); + object_info->metadata = object_storage->getObjectMetadata(path); + } + } + while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); + + QueryPipelineBuilder builder; + std::shared_ptr source; + std::unique_ptr read_buf; + + std::optional num_rows_from_cache = need_only_count + && getContext()->getSettingsRef().use_cache_for_count_from_files + ? tryGetNumRowsFromCache(*object_info) + : std::nullopt; + + if (num_rows_from_cache) + { + /// We should not return single chunk with all number of rows, + /// because there is a chance that this chunk will be materialized later + /// (it can cause memory problems even with default values in columns or when virtual columns are requested). + /// Instead, we use special ConstChunkGenerator that will generate chunks + /// with max_block_size rows until total number of rows is reached. + builder.init(Pipe(std::make_shared( + read_from_format_info.format_header, *num_rows_from_cache, max_block_size))); + } + else + { + CompressionMethod compression_method; + if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) + { + compression_method = chooseCompressionMethod(configuration->getPathInArchive(), configuration->compression_method); + const auto & archive_reader = object_info_in_archive->archive_reader; + read_buf = archive_reader->readFile(object_info_in_archive->path_in_archive, /*throw_on_not_found=*/true); + } + else + { + compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); + read_buf = createReadBuffer(*object_info); + } + + auto input_format = FormatFactory::instance().getInput( + configuration->format, + *read_buf, + read_from_format_info.format_header, + getContext(), + max_block_size, + format_settings, + need_only_count ? 1 : max_parsing_threads, + std::nullopt, + true/* is_remote_fs */, + compression_method, + need_only_count); + + if (key_condition) + input_format->setKeyCondition(key_condition); + + if (need_only_count) + input_format->needOnlyCount(); + + builder.init(Pipe(input_format)); + + if (columns_desc.hasDefaults()) + { + builder.addSimpleTransform( + [&](const Block & header) + { + return std::make_shared(header, columns_desc, *input_format, getContext()); + }); + } + + source = input_format; + } + + /// Add ExtractColumnsTransform to extract requested columns/subcolumns + /// from chunk read by IInputFormat. + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, read_from_format_info.requested_columns); + }); + + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + auto current_reader = std::make_unique(*pipeline); + + ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); + + return ReaderHolder( + object_info, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)); +} + +std::future StorageObjectStorageSource::createReaderAsync(size_t processor) +{ + return create_reader_scheduler([=, this] { return createReader(processor); }, Priority{}); +} + +std::unique_ptr StorageObjectStorageSource::createReadBuffer(const ObjectInfo & object_info) +{ + const auto & object_size = object_info.metadata->size_bytes; + + auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); + read_settings.enable_filesystem_cache = false; + /// FIXME: Changing this setting to default value breaks something around parquet reading + read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; + + const bool object_too_small = object_size <= 2 * getContext()->getSettings().max_download_buffer_size; + const bool use_prefetch = object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; + read_settings.remote_fs_method = use_prefetch ? RemoteFSReadMethod::threadpool : RemoteFSReadMethod::read; + /// User's object may change, don't cache it. + read_settings.use_page_cache_for_disks_without_file_cache = false; + + // Create a read buffer that will prefetch the first ~1 MB of the file. + // When reading lots of tiny files, this prefetching almost doubles the throughput. + // For bigger files, parallel reading is more useful. + if (use_prefetch) + { + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + + auto async_reader = object_storage->readObjects( + StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); + + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + + return async_reader; + } + else + { + /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. + return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + } +} + +StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name_) + : logger(getLogger(logger_name_)) +{ +} + +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::IIterator::next(size_t processor) +{ + auto object_info = nextImpl(processor); + + if (object_info) + { + LOG_TEST(logger, "Next key: {}", object_info->getFileName()); + } + + return object_info; +} + +StorageObjectStorageSource::GlobIterator::GlobIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns_, + ContextPtr context_, + ObjectInfos * read_keys_, + size_t list_object_keys_size, + bool throw_on_zero_files_match_, + std::function file_progress_callback_) + : IIterator("GlobIterator") + , WithContext(context_) + , object_storage(object_storage_) + , configuration(configuration_) + , virtual_columns(virtual_columns_) + , throw_on_zero_files_match(throw_on_zero_files_match_) + , read_keys(read_keys_) + , file_progress_callback(file_progress_callback_) +{ + if (configuration->isNamespaceWithGlobs()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside namespace name"); + } + else if (configuration->isPathWithGlobs()) + { + const auto key_with_globs = configuration_->getPath(); + const auto key_prefix = configuration->getPathWithoutGlobs(); + object_storage_iterator = object_storage->iterate(key_prefix, list_object_keys_size); + + matcher = std::make_unique(makeRegexpPatternFromGlobs(key_with_globs)); + if (!matcher->ok()) + { + throw Exception( + ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", key_with_globs, matcher->error()); + } + + recursive = key_with_globs == "/**"; + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Using glob iterator with path without globs is not allowed (used path: {})", + configuration->getPath()); + } +} + +size_t StorageObjectStorageSource::GlobIterator::estimatedKeysCount() +{ + if (object_infos.empty() && !is_finished && object_storage_iterator->isValid()) + { + /// 1000 files were listed, and we cannot make any estimation of _how many more_ there are (because we list bucket lazily); + /// If there are more objects in the bucket, limiting the number of streams is the last thing we may want to do + /// as it would lead to serious slow down of the execution, since objects are going + /// to be fetched sequentially rather than in-parallel with up to times. + return std::numeric_limits::max(); + } + return object_infos.size(); +} + +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImpl(size_t processor) +{ + std::lock_guard lock(next_mutex); + auto object_info = nextImplUnlocked(processor); + if (first_iteration && !object_info && throw_on_zero_files_match) + { + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, + "Can not match any files with path {}", + configuration->getPath()); + } + first_iteration = false; + return object_info; +} + +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImplUnlocked(size_t /* processor */) +{ + bool current_batch_processed = object_infos.empty() || index >= object_infos.size(); + if (is_finished && current_batch_processed) + return {}; + + if (current_batch_processed) + { + ObjectInfos new_batch; + while (new_batch.empty()) + { + auto result = object_storage_iterator->getCurrentBatchAndScheduleNext(); + if (!result.has_value()) + { + is_finished = true; + return {}; + } + + new_batch = std::move(result.value()); + for (auto it = new_batch.begin(); it != new_batch.end();) + { + if (!recursive && !re2::RE2::FullMatch((*it)->getPath(), *matcher)) + it = new_batch.erase(it); + else + ++it; + } + } + + index = 0; + + if (filter_dag) + { + std::vector paths; + paths.reserve(new_batch.size()); + for (const auto & object_info : new_batch) + { + chassert(object_info); + paths.push_back(getUniqueStoragePathIdentifier(*configuration, *object_info, false)); + } + + VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); + LOG_TEST(logger, "Filtered files: {} -> {}", paths.size(), new_batch.size()); + } + + if (read_keys) + read_keys->insert(read_keys->end(), new_batch.begin(), new_batch.end()); + + object_infos = std::move(new_batch); + + if (file_progress_callback) + { + for (const auto & object_info : object_infos) + { + chassert(object_info->metadata); + file_progress_callback(FileProgress(0, object_info->metadata->size_bytes)); + } + } + } + + if (index >= object_infos.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Index out of bound for blob metadata"); + + return object_infos[index++]; +} + +StorageObjectStorageSource::KeysIterator::KeysIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const NamesAndTypesList & virtual_columns_, + ObjectInfos * read_keys_, + bool ignore_non_existent_files_, + std::function file_progress_callback_) + : IIterator("KeysIterator") + , object_storage(object_storage_) + , configuration(configuration_) + , virtual_columns(virtual_columns_) + , file_progress_callback(file_progress_callback_) + , keys(configuration->getPaths()) + , ignore_non_existent_files(ignore_non_existent_files_) +{ + if (read_keys_) + { + /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? + for (auto && key : keys) + { + auto object_info = std::make_shared(key); + read_keys_->emplace_back(object_info); + } + } +} + +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::KeysIterator::nextImpl(size_t /* processor */) +{ + while (true) + { + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= keys.size()) + return {}; + + auto key = keys[current_index]; + + ObjectMetadata object_metadata{}; + if (ignore_non_existent_files) + { + auto metadata = object_storage->tryGetObjectMetadata(key); + if (!metadata) + continue; + } + else + object_metadata = object_storage->getObjectMetadata(key); + + if (file_progress_callback) + file_progress_callback(FileProgress(0, object_metadata.size_bytes)); + + return std::make_shared(key, object_metadata); + } +} + +StorageObjectStorageSource::ReaderHolder::ReaderHolder( + ObjectInfoPtr object_info_, + std::unique_ptr read_buf_, + std::shared_ptr source_, + std::unique_ptr pipeline_, + std::unique_ptr reader_) + : object_info(std::move(object_info_)) + , read_buf(std::move(read_buf_)) + , source(std::move(source_)) + , pipeline(std::move(pipeline_)) + , reader(std::move(reader_)) +{ +} + +StorageObjectStorageSource::ReaderHolder & +StorageObjectStorageSource::ReaderHolder::operator=(ReaderHolder && other) noexcept +{ + /// The order of destruction is important. + /// reader uses pipeline, pipeline uses read_buf. + reader = std::move(other.reader); + pipeline = std::move(other.pipeline); + source = std::move(other.source); + read_buf = std::move(other.read_buf); + object_info = std::move(other.object_info); + return *this; +} + +StorageObjectStorageSource::ReadTaskIterator::ReadTaskIterator( + const ReadTaskCallback & callback_, size_t max_threads_count) + : IIterator("ReadTaskIterator") + , callback(callback_) +{ + ThreadPool pool( + CurrentMetrics::StorageObjectStorageThreads, + CurrentMetrics::StorageObjectStorageThreadsActive, + CurrentMetrics::StorageObjectStorageThreadsScheduled, max_threads_count); + + auto pool_scheduler = threadPoolCallbackRunnerUnsafe(pool, "ReadTaskIter"); + + std::vector> keys; + keys.reserve(max_threads_count); + for (size_t i = 0; i < max_threads_count; ++i) + keys.push_back(pool_scheduler([this] { return callback(); }, Priority{})); + + pool.wait(); + buffer.reserve(max_threads_count); + for (auto & key_future : keys) + { + auto key = key_future.get(); + if (!key.empty()) + buffer.emplace_back(std::make_shared(key, std::nullopt)); + } +} + +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::ReadTaskIterator::nextImpl(size_t) +{ + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= buffer.size()) + return std::make_shared(callback()); + + return buffer[current_index]; +} + +static IArchiveReader::NameFilter createArchivePathFilter(const std::string & archive_pattern) +{ + auto matcher = std::make_shared(makeRegexpPatternFromGlobs(archive_pattern)); + if (!matcher->ok()) + { + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", + archive_pattern, matcher->error()); + } + return [matcher](const std::string & p) mutable { return re2::RE2::FullMatch(p, *matcher); }; +} + +StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive::ObjectInfoInArchive( + ObjectInfoPtr archive_object_, + const std::string & path_in_archive_, + std::shared_ptr archive_reader_) + : archive_object(archive_object_) + , path_in_archive(path_in_archive_) + , archive_reader(archive_reader_) +{ +} + +StorageObjectStorageSource::ArchiveIterator::ArchiveIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + std::unique_ptr archives_iterator_, + ContextPtr context_, + ObjectInfos * read_keys_) + : IIterator("ArchiveIterator") + , WithContext(context_) + , object_storage(object_storage_) + , is_path_in_archive_with_globs(configuration_->isPathInArchiveWithGlobs()) + , archives_iterator(std::move(archives_iterator_)) + , filter(is_path_in_archive_with_globs ? createArchivePathFilter(configuration_->getPathInArchive()) : IArchiveReader::NameFilter{}) + , path_in_archive(is_path_in_archive_with_globs ? "" : configuration_->getPathInArchive()) + , read_keys(read_keys_) +{ +} + +std::shared_ptr +StorageObjectStorageSource::ArchiveIterator::createArchiveReader(ObjectInfoPtr object_info) const +{ + const auto size = object_info->metadata->size_bytes; + return DB::createArchiveReader( + /* path_to_archive */object_info->getPath(), + /* archive_read_function */[=, this]() + { + StoredObject stored_object(object_info->getPath(), "", size); + return object_storage->readObject(stored_object, getContext()->getReadSettings()); + }, + /* archive_size */size); +} + +StorageObjectStorageSource::ObjectInfoPtr +StorageObjectStorageSource::ArchiveIterator::nextImpl(size_t processor) +{ + std::unique_lock lock{next_mutex}; + while (true) + { + if (filter) + { + if (!file_enumerator) + { + archive_object = archives_iterator->next(processor); + if (!archive_object) + return {}; + + archive_reader = createArchiveReader(archive_object); + file_enumerator = archive_reader->firstFile(); + if (!file_enumerator) + continue; + } + else if (!file_enumerator->nextFile()) + { + file_enumerator.reset(); + continue; + } + + path_in_archive = file_enumerator->getFileName(); + if (!filter(path_in_archive)) + continue; + } + else + { + archive_object = archives_iterator->next(processor); + if (!archive_object) + return {}; + + if (!archive_object->metadata) + archive_object->metadata = object_storage->getObjectMetadata(archive_object->getPath()); + + archive_reader = createArchiveReader(archive_object); + if (!archive_reader->fileExists(path_in_archive)) + continue; + } + + auto object_in_archive = std::make_shared(archive_object, path_in_archive, archive_reader); + + if (read_keys != nullptr) + read_keys->push_back(object_in_archive); + + return object_in_archive; + } +} + +size_t StorageObjectStorageSource::ArchiveIterator::estimatedKeysCount() +{ + return archives_iterator->estimatedKeysCount(); +} + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h new file mode 100644 index 00000000000..fd7c7aa7102 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -0,0 +1,310 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class SchemaCache; + +class StorageObjectStorageSource : public SourceWithKeyCondition, WithContext +{ + friend class StorageS3QueueSource; +public: + using Configuration = StorageObjectStorage::Configuration; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + using ObjectInfo = StorageObjectStorage::ObjectInfo; + using ObjectInfos = StorageObjectStorage::ObjectInfos; + using ObjectInfoPtr = StorageObjectStorage::ObjectInfoPtr; + + class IIterator; + class ReadTaskIterator; + class GlobIterator; + class KeysIterator; + class ArchiveIterator; + + StorageObjectStorageSource( + String name_, + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration, + const ReadFromFormatInfo & info, + const std::optional & format_settings_, + ContextPtr context_, + UInt64 max_block_size_, + std::shared_ptr file_iterator_, + size_t max_parsing_threads_, + bool need_only_count_); + + ~StorageObjectStorageSource() override; + + String getName() const override { return name; } + + void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override; + + Chunk generate() override; + + static std::shared_ptr createFileIterator( + ConfigurationPtr configuration, + ObjectStoragePtr object_storage, + bool distributed_processing, + const ContextPtr & local_context, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns, + ObjectInfos * read_keys, + std::function file_progress_callback = {}); + + static std::string getUniqueStoragePathIdentifier( + const Configuration & configuration, + const ObjectInfo & object_info, + bool include_connection_info = true); + +protected: + const String name; + ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; + const std::optional format_settings; + const UInt64 max_block_size; + const bool need_only_count; + const size_t max_parsing_threads; + const ReadFromFormatInfo read_from_format_info; + const std::shared_ptr create_reader_pool; + + ColumnsDescription columns_desc; + std::shared_ptr file_iterator; + SchemaCache & schema_cache; + bool initialized = false; + size_t total_rows_in_file = 0; + LoggerPtr log = getLogger("StorageObjectStorageSource"); + + struct ReaderHolder : private boost::noncopyable + { + public: + ReaderHolder( + ObjectInfoPtr object_info_, + std::unique_ptr read_buf_, + std::shared_ptr source_, + std::unique_ptr pipeline_, + std::unique_ptr reader_); + + ReaderHolder() = default; + ReaderHolder(ReaderHolder && other) noexcept { *this = std::move(other); } + ReaderHolder & operator=(ReaderHolder && other) noexcept; + + explicit operator bool() const { return reader != nullptr; } + PullingPipelineExecutor * operator->() { return reader.get(); } + const PullingPipelineExecutor * operator->() const { return reader.get(); } + + const ObjectInfo & getObjectInfo() const { return *object_info; } + const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } + + private: + ObjectInfoPtr object_info; + std::unique_ptr read_buf; + std::shared_ptr source; + std::unique_ptr pipeline; + std::unique_ptr reader; + }; + + ReaderHolder reader; + ThreadPoolCallbackRunnerUnsafe create_reader_scheduler; + std::future reader_future; + + /// Recreate ReadBuffer and Pipeline for each file. + ReaderHolder createReader(size_t processor = 0); + std::future createReaderAsync(size_t processor = 0); + std::unique_ptr createReadBuffer(const ObjectInfo & object_info); + + void addNumRowsToCache(const ObjectInfo & object_info, size_t num_rows); + std::optional tryGetNumRowsFromCache(const ObjectInfo & object_info); + void lazyInitialize(size_t processor); +}; + +class StorageObjectStorageSource::IIterator +{ +public: + explicit IIterator(const std::string & logger_name_); + + virtual ~IIterator() = default; + + virtual size_t estimatedKeysCount() = 0; + + ObjectInfoPtr next(size_t processor); + +protected: + virtual ObjectInfoPtr nextImpl(size_t processor) = 0; + LoggerPtr logger; +}; + +class StorageObjectStorageSource::ReadTaskIterator : public IIterator +{ +public: + ReadTaskIterator(const ReadTaskCallback & callback_, size_t max_threads_count); + + size_t estimatedKeysCount() override { return buffer.size(); } + +private: + ObjectInfoPtr nextImpl(size_t) override; + + ReadTaskCallback callback; + ObjectInfos buffer; + std::atomic_size_t index = 0; +}; + +class StorageObjectStorageSource::GlobIterator : public IIterator, WithContext +{ +public: + GlobIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns_, + ContextPtr context_, + ObjectInfos * read_keys_, + size_t list_object_keys_size, + bool throw_on_zero_files_match_, + std::function file_progress_callback_ = {}); + + ~GlobIterator() override = default; + + size_t estimatedKeysCount() override; + +private: + ObjectInfoPtr nextImpl(size_t processor) override; + ObjectInfoPtr nextImplUnlocked(size_t processor); + void createFilterAST(const String & any_key); + void fillBufferForKey(const std::string & uri_key); + + const ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; + const NamesAndTypesList virtual_columns; + const bool throw_on_zero_files_match; + + size_t index = 0; + + ObjectInfos object_infos; + ObjectInfos * read_keys; + ActionsDAGPtr filter_dag; + ObjectStorageIteratorPtr object_storage_iterator; + bool recursive{false}; + std::vector expanded_keys; + std::vector::iterator expanded_keys_iter; + + std::unique_ptr matcher; + + bool is_finished = false; + bool first_iteration = true; + std::mutex next_mutex; + + std::function file_progress_callback; +}; + +class StorageObjectStorageSource::KeysIterator : public IIterator +{ +public: + KeysIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const NamesAndTypesList & virtual_columns_, + ObjectInfos * read_keys_, + bool ignore_non_existent_files_, + std::function file_progress_callback = {}); + + ~KeysIterator() override = default; + + size_t estimatedKeysCount() override { return keys.size(); } + +private: + ObjectInfoPtr nextImpl(size_t processor) override; + + const ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; + const NamesAndTypesList virtual_columns; + const std::function file_progress_callback; + const std::vector keys; + std::atomic index = 0; + bool ignore_non_existent_files; +}; + +/* + * An archives iterator. + * Allows to iterate files inside one or many archives. + * `archives_iterator` is an iterator which iterates over different archives. + * There are two ways to read files in archives: + * 1. When we want to read one concete file in each archive. + * In this case we go through all archives, check if this certain file + * exists within this archive and read it if it exists. + * 2. When we have a certain pattern of files we want to read in each archive. + * For this purpose we create a filter defined as IArchiveReader::NameFilter. + */ +class StorageObjectStorageSource::ArchiveIterator : public IIterator, private WithContext +{ +public: + explicit ArchiveIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + std::unique_ptr archives_iterator_, + ContextPtr context_, + ObjectInfos * read_keys_); + + size_t estimatedKeysCount() override; + + struct ObjectInfoInArchive : public ObjectInfo + { + ObjectInfoInArchive( + ObjectInfoPtr archive_object_, + const std::string & path_in_archive_, + std::shared_ptr archive_reader_); + + std::string getFileName() const override + { + return path_in_archive; + } + + std::string getPath() const override + { + return archive_object->getPath() + "::" + path_in_archive; + } + + std::string getPathToArchive() const override + { + return archive_object->getPath(); + } + + bool isArchive() const override { return true; } + + const ObjectInfoPtr archive_object; + const std::string path_in_archive; + const std::shared_ptr archive_reader; + }; + +private: + ObjectInfoPtr nextImpl(size_t processor) override; + std::shared_ptr createArchiveReader(ObjectInfoPtr object_info) const; + + const ObjectStoragePtr object_storage; + const bool is_path_in_archive_with_globs; + /// Iterator which iterates through different archives. + const std::unique_ptr archives_iterator; + /// Used when files inside archive are defined with a glob + const IArchiveReader::NameFilter filter = {}; + /// Current file inside the archive. + std::string path_in_archive = {}; + /// Read keys of files inside archives. + ObjectInfos * read_keys; + /// Object pointing to archive (NOT path within archive). + ObjectInfoPtr archive_object; + /// Reader of the archive. + std::shared_ptr archive_reader; + /// File enumerator inside the archive. + std::unique_ptr file_enumerator; + + std::mutex next_mutex; +}; + +} diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp new file mode 100644 index 00000000000..e49e14d2a0c --- /dev/null +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -0,0 +1,76 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +std::optional checkAndGetNewFileOnInsertIfNeeded( + const IObjectStorage & object_storage, + const StorageObjectStorage::Configuration & configuration, + const StorageObjectStorage::QuerySettings & settings, + const String & key, + size_t sequence_number) +{ + if (settings.truncate_on_insert + || !object_storage.exists(StoredObject(key))) + return std::nullopt; + + if (settings.create_new_file_on_insert) + { + auto pos = key.find_first_of('.'); + String new_key; + do + { + new_key = key.substr(0, pos) + "." + std::to_string(sequence_number) + (pos == std::string::npos ? "" : key.substr(pos)); + ++sequence_number; + } + while (object_storage.exists(StoredObject(new_key))); + + return new_key; + } + + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Object in bucket {} with key {} already exists. " + "If you want to overwrite it, enable setting {}_truncate_on_insert, if you " + "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", + configuration.getNamespace(), key, configuration.getTypeName(), configuration.getTypeName()); +} + +void resolveSchemaAndFormat( + ColumnsDescription & columns, + std::string & format, + ObjectStoragePtr object_storage, + const StorageObjectStorage::ConfigurationPtr & configuration, + std::optional format_settings, + const ContextPtr & context) +{ + if (columns.empty()) + { + if (format == "auto") + std::tie(columns, format) = + StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, context); + else + columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, context); + } + else if (format == "auto") + { + format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, context); + } + + if (!columns.hasOnlyOrdinary()) + { + /// We don't allow special columns. + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Special columns are not supported for {} storage" + "like MATERIALIZED, ALIAS or EPHEMERAL", configuration->getTypeName()); + } +} + +} diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h new file mode 100644 index 00000000000..2077999df41 --- /dev/null +++ b/src/Storages/ObjectStorage/Utils.h @@ -0,0 +1,24 @@ +#pragma once +#include "StorageObjectStorage.h" + +namespace DB +{ + +class IObjectStorage; + +std::optional checkAndGetNewFileOnInsertIfNeeded( + const IObjectStorage & object_storage, + const StorageObjectStorage::Configuration & configuration, + const StorageObjectStorage::QuerySettings & settings, + const std::string & key, + size_t sequence_number); + +void resolveSchemaAndFormat( + ColumnsDescription & columns, + std::string & format, + ObjectStoragePtr object_storage, + const StorageObjectStorage::ConfigurationPtr & configuration, + std::optional format_settings, + const ContextPtr & context); + +} diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp new file mode 100644 index 00000000000..bf595b2f5d4 --- /dev/null +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -0,0 +1,157 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +static std::shared_ptr createStorageObjectStorage( + const StorageFactory::Arguments & args, + StorageObjectStorage::ConfigurationPtr configuration, + ContextPtr context) +{ + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, context, false); + + // Use format settings from global server context + settings from + // the SETTINGS clause of the create query. Settings from current + // session and user are ignored. + std::optional format_settings; + if (args.storage_def->settings) + { + FormatFactorySettings user_format_settings; + + // Apply changed settings from global context, but ignore the + // unknown ones, because we only have the format settings here. + const auto & changes = context->getSettingsRef().changes(); + for (const auto & change : changes) + { + if (user_format_settings.has(change.name)) + user_format_settings.set(change.name, change.value); + } + + // Apply changes from SETTINGS clause, with validation. + user_format_settings.applyChanges(args.storage_def->settings->changes); + format_settings = getFormatSettings(context, user_format_settings); + } + else + { + format_settings = getFormatSettings(context); + } + + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); + + return std::make_shared( + configuration, + configuration->createObjectStorage(context, /* is_readonly */false), + args.getContext(), + args.table_id, + args.columns, + args.constraints, + args.comment, + format_settings, + /* distributed_processing */ false, + partition_by); +} + +#endif + +#if USE_AZURE_BLOB_STORAGE +void registerStorageAzure(StorageFactory & factory) +{ + factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::AZURE, + }); +} +#endif + +#if USE_AWS_S3 +void registerStorageS3Impl(const String & name, StorageFactory & factory) +{ + factory.registerStorage(name, [=](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +void registerStorageS3(StorageFactory & factory) +{ + registerStorageS3Impl("S3", factory); +} + +void registerStorageCOS(StorageFactory & factory) +{ + registerStorageS3Impl("COSN", factory); +} + +void registerStorageOSS(StorageFactory & factory) +{ + registerStorageS3Impl("OSS", factory); +} + +#endif + +#if USE_HDFS +void registerStorageHDFS(StorageFactory & factory) +{ + factory.registerStorage("HDFS", [=](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::HDFS, + }); +} +#endif + +void registerStorageObjectStorage(StorageFactory & factory) +{ +#if USE_AWS_S3 + registerStorageS3(factory); + registerStorageCOS(factory); + registerStorageOSS(factory); +#endif +#if USE_AZURE_BLOB_STORAGE + registerStorageAzure(factory); +#endif +#if USE_HDFS + registerStorageHDFS(factory); +#endif + UNUSED(factory); +} + +} diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index b5bee2cc8da..c8aaece0711 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -5,9 +5,9 @@ #include #include #include -#include #include #include +#include namespace CurrentMetrics @@ -31,11 +31,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( +StorageS3QueueSource::S3QueueObjectInfo::S3QueueObjectInfo( const std::string & key_, - std::optional info_, + const ObjectMetadata & object_metadata_, Metadata::ProcessingNodeHolderPtr processing_holder_) - : StorageS3Source::KeyWithInfo(key_, info_) + : ObjectInfo(key_, object_metadata_) , processing_holder(processing_holder_) { } @@ -46,7 +46,8 @@ StorageS3QueueSource::FileIterator::FileIterator( size_t current_shard_, std::atomic & shutdown_called_, LoggerPtr logger_) - : metadata(metadata_) + : StorageObjectStorageSource::IIterator("S3QueueIterator") + , metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) , shutdown_called(shutdown_called_) , log(logger_) @@ -56,15 +57,15 @@ StorageS3QueueSource::FileIterator::FileIterator( if (sharded_processing) { for (const auto & id : metadata->getProcessingIdsForShard(current_shard)) - sharded_keys.emplace(id, std::deque{}); + sharded_keys.emplace(id, std::deque{}); } } -StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next(size_t idx) +StorageS3QueueSource::ObjectInfoPtr StorageS3QueueSource::FileIterator::nextImpl(size_t processor) { while (!shutdown_called) { - KeyWithInfoPtr val{nullptr}; + ObjectInfoPtr val{nullptr}; { std::unique_lock lk(sharded_keys_mutex, std::defer_lock); @@ -74,36 +75,36 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next(si /// we need to check sharded_keys and to next() under lock. lk.lock(); - if (auto it = sharded_keys.find(idx); it != sharded_keys.end()) + if (auto it = sharded_keys.find(processor); it != sharded_keys.end()) { auto & keys = it->second; if (!keys.empty()) { val = keys.front(); keys.pop_front(); - chassert(idx == metadata->getProcessingIdForPath(val->key)); + chassert(processor == metadata->getProcessingIdForPath(val->relative_path)); } } else { throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing id {} does not exist (Expected ids: {})", - idx, fmt::join(metadata->getProcessingIdsForShard(current_shard), ", ")); + processor, fmt::join(metadata->getProcessingIdsForShard(current_shard), ", ")); } } if (!val) { - val = glob_iterator->next(); + val = glob_iterator->next(processor); if (val && sharded_processing) { - const auto processing_id_for_key = metadata->getProcessingIdForPath(val->key); - if (idx != processing_id_for_key) + const auto processing_id_for_key = metadata->getProcessingIdForPath(val->relative_path); + if (processor != processing_id_for_key) { if (metadata->isProcessingIdBelongsToShard(processing_id_for_key, current_shard)) { LOG_TEST(log, "Putting key {} into queue of processor {} (total: {})", - val->key, processing_id_for_key, sharded_keys.size()); + val->relative_path, processing_id_for_key, sharded_keys.size()); if (auto it = sharded_keys.find(processing_id_for_key); it != sharded_keys.end()) { @@ -131,25 +132,25 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next(si return {}; } - auto processing_holder = metadata->trySetFileAsProcessing(val->key); + auto processing_holder = metadata->trySetFileAsProcessing(val->relative_path); if (shutdown_called) { LOG_TEST(log, "Shutdown was called, stopping file iterator"); return {}; } - LOG_TEST(log, "Checking if can process key {} for processing_id {}", val->key, idx); + LOG_TEST(log, "Checking if can process key {} for processing_id {}", val->relative_path, processor); if (processing_holder) { - return std::make_shared(val->key, val->info, processing_holder); + return std::make_shared(val->relative_path, val->metadata.value(), processing_holder); } else if (sharded_processing - && metadata->getFileStatus(val->key)->state == S3QueueFilesMetadata::FileStatus::State::Processing) + && metadata->getFileStatus(val->relative_path)->state == S3QueueFilesMetadata::FileStatus::State::Processing) { throw Exception(ErrorCodes::LOGICAL_ERROR, "File {} is processing by someone else in sharded processing. " - "It is a bug", val->key); + "It is a bug", val->relative_path); } } return {}; @@ -163,7 +164,7 @@ size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() StorageS3QueueSource::StorageS3QueueSource( String name_, const Block & header_, - std::unique_ptr internal_source_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, size_t processing_id_, const S3QueueAction & action_, @@ -192,11 +193,6 @@ StorageS3QueueSource::StorageS3QueueSource( { } -StorageS3QueueSource::~StorageS3QueueSource() -{ - internal_source->create_reader_pool.wait(); -} - String StorageS3QueueSource::getName() const { return name; @@ -223,7 +219,7 @@ Chunk StorageS3QueueSource::generate() if (!reader) break; - const auto * key_with_info = dynamic_cast(&reader.getKeyWithInfo()); + const auto * key_with_info = dynamic_cast(&reader.getObjectInfo()); auto file_status = key_with_info->processing_holder->getFileStatus(); if (isCancelled()) @@ -239,15 +235,17 @@ Chunk StorageS3QueueSource::generate() catch (...) { LOG_ERROR(log, "Failed to set file {} as failed: {}", - key_with_info->key, getCurrentExceptionMessage(true)); + key_with_info->relative_path, getCurrentExceptionMessage(true)); } - appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); + appendLogElement(reader.getObjectInfo().getPath(), *file_status, processed_rows_from_file, false); } break; } + const auto & path = reader.getObjectInfo().getPath(); + if (shutdown_called) { if (processed_rows_from_file == 0) @@ -257,7 +255,7 @@ Chunk StorageS3QueueSource::generate() { LOG_DEBUG( log, "Table is being dropped, {} rows are already processed from {}, but file is not fully processed", - processed_rows_from_file, reader.getFile()); + processed_rows_from_file, path); try { @@ -266,10 +264,10 @@ Chunk StorageS3QueueSource::generate() catch (...) { LOG_ERROR(log, "Failed to set file {} as failed: {}", - key_with_info->key, getCurrentExceptionMessage(true)); + key_with_info->relative_path, getCurrentExceptionMessage(true)); } - appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); + appendLogElement(path, *file_status, processed_rows_from_file, false); /// Leave the file half processed. Table is being dropped, so we do not care. break; @@ -277,7 +275,7 @@ Chunk StorageS3QueueSource::generate() LOG_DEBUG(log, "Shutdown called, but file {} is partially processed ({} rows). " "Will process the file fully and then shutdown", - reader.getFile(), processed_rows_from_file); + path, processed_rows_from_file); } auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); @@ -291,30 +289,31 @@ Chunk StorageS3QueueSource::generate() Chunk chunk; if (reader->pull(chunk)) { - LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getPath()); + LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), path); file_status->processed_rows += chunk.getNumRows(); processed_rows_from_file += chunk.getNumRows(); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath(), reader.getKeyWithInfo().info->size); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( + chunk, requested_virtual_columns, path, reader.getObjectInfo().metadata->size_bytes); return chunk; } } catch (...) { const auto message = getCurrentExceptionMessage(true); - LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getFile(), message); + LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", path, message); files_metadata->setFileFailed(key_with_info->processing_holder, message); - appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); + appendLogElement(path, *file_status, processed_rows_from_file, false); throw; } files_metadata->setFileProcessed(key_with_info->processing_holder); - applyActionAfterProcessing(reader.getFile()); + applyActionAfterProcessing(path); - appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, true); + appendLogElement(path, *file_status, processed_rows_from_file, true); file_status.reset(); processed_rows_from_file = 0; @@ -330,11 +329,11 @@ Chunk StorageS3QueueSource::generate() if (!reader) break; - file_status = files_metadata->getFileStatus(reader.getFile()); + file_status = files_metadata->getFileStatus(reader.getObjectInfo().getPath()); /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. - internal_source->create_reader_pool.wait(); + internal_source->create_reader_pool->wait(); reader_future = internal_source->createReaderAsync(processing_id); } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index a657459ed9d..663577e055b 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -5,7 +5,8 @@ #include #include #include -#include +#include +#include #include @@ -14,28 +15,35 @@ namespace Poco { class Logger; } namespace DB { +struct ObjectMetadata; + class StorageS3QueueSource : public ISource, WithContext { public: - using IIterator = StorageS3Source::IIterator; - using KeyWithInfoPtr = StorageS3Source::KeyWithInfoPtr; - using GlobIterator = StorageS3Source::DisclosedGlobIterator; + using Storage = StorageObjectStorage; + + using ConfigurationPtr = Storage::ConfigurationPtr; + using GlobIterator = StorageObjectStorageSource::GlobIterator; using ZooKeeperGetter = std::function; using RemoveFileFunc = std::function; using FileStatusPtr = S3QueueFilesMetadata::FileStatusPtr; + using ReaderHolder = StorageObjectStorageSource::ReaderHolder; using Metadata = S3QueueFilesMetadata; + using ObjectInfo = StorageObjectStorageSource::ObjectInfo; + using ObjectInfoPtr = std::shared_ptr; + using ObjectInfos = std::vector; - struct S3QueueKeyWithInfo : public StorageS3Source::KeyWithInfo + struct S3QueueObjectInfo : public ObjectInfo { - S3QueueKeyWithInfo( - const std::string & key_, - std::optional info_, - Metadata::ProcessingNodeHolderPtr processing_holder_); + S3QueueObjectInfo( + const std::string & key_, + const ObjectMetadata & object_metadata_, + Metadata::ProcessingNodeHolderPtr processing_holder_); Metadata::ProcessingNodeHolderPtr processing_holder; }; - class FileIterator : public IIterator + class FileIterator : public StorageObjectStorageSource::IIterator { public: FileIterator( @@ -48,7 +56,7 @@ public: /// Note: /// List results in s3 are always returned in UTF-8 binary order. /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) - KeyWithInfoPtr next(size_t idx) override; + ObjectInfoPtr nextImpl(size_t processor) override; size_t estimatedKeysCount() override; @@ -61,14 +69,14 @@ public: const bool sharded_processing; const size_t current_shard; - std::unordered_map> sharded_keys; + std::unordered_map> sharded_keys; std::mutex sharded_keys_mutex; }; StorageS3QueueSource( String name_, const Block & header_, - std::unique_ptr internal_source_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, size_t processing_id_, const S3QueueAction & action_, @@ -81,8 +89,6 @@ public: const StorageID & storage_id_, LoggerPtr log_); - ~StorageS3QueueSource() override; - static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); String getName() const override; @@ -94,7 +100,7 @@ private: const S3QueueAction action; const size_t processing_id; const std::shared_ptr files_metadata; - const std::shared_ptr internal_source; + const std::shared_ptr internal_source; const NamesAndTypesList requested_virtual_columns; const std::atomic & shutdown_called; const std::atomic & table_is_being_dropped; @@ -104,15 +110,14 @@ private: RemoveFileFunc remove_file_func; LoggerPtr log; - using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; std::future reader_future; std::atomic initialized{false}; size_t processed_rows_from_file = 0; - void lazyInitialize(); void applyActionAfterProcessing(const String & path); void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); + void lazyInitialize(); }; } diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 1830bac4743..f0b7568ae7f 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB @@ -33,7 +33,7 @@ namespace S3QueueTableMetadata::S3QueueTableMetadata( - const StorageS3::Configuration & configuration, + const StorageObjectStorage::Configuration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata) { diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 84087f72a6a..bb8f8ccf2c4 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -3,7 +3,8 @@ #if USE_AWS_S3 #include -#include +#include +#include #include namespace DB @@ -27,7 +28,10 @@ struct S3QueueTableMetadata UInt64 s3queue_processing_threads_num = 1; S3QueueTableMetadata() = default; - S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata); + S3QueueTableMetadata( + const StorageObjectStorage::Configuration & configuration, + const S3QueueSettings & engine_settings, + const StorageInMemoryMetadata & storage_metadata); void read(const String & metadata_str); static S3QueueTableMetadata parse(const String & metadata_str); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 16e42e32b8a..f8eb288921c 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -25,6 +25,8 @@ #include #include #include +#include +#include #include #include @@ -52,11 +54,6 @@ namespace ErrorCodes namespace { - bool containsGlobs(const S3::URI & url) - { - return url.key.find_first_of("*?{") != std::string::npos; - } - std::string chooseZooKeeperPath(const StorageID & table_id, const Settings & settings, const S3QueueSettings & s3queue_settings) { std::string zk_path_prefix = settings.s3queue_default_zookeeper_path.value; @@ -100,7 +97,7 @@ namespace StorageS3Queue::StorageS3Queue( std::unique_ptr s3queue_settings_, - const StorageS3::Configuration & configuration_, + const ConfigurationPtr configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -119,15 +116,15 @@ StorageS3Queue::StorageS3Queue( , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , log(getLogger("StorageS3Queue (" + table_id_.getFullTableName() + ")")) { - if (configuration.url.key.empty()) + if (configuration->getPath().empty()) { - configuration.url.key = "/*"; + configuration->setPath("/*"); } - else if (configuration.url.key.ends_with('/')) + else if (configuration->getPath().ends_with('/')) { - configuration.url.key += '*'; + configuration->setPath(configuration->getPath() + '*'); } - else if (!containsGlobs(configuration.url)) + else if (!configuration->isPathWithGlobs()) { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); } @@ -142,31 +139,20 @@ StorageS3Queue::StorageS3Queue( checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef()); - configuration.update(context_); - FormatFactory::instance().checkFormatName(configuration.format); - context_->getRemoteHostFilter().checkURL(configuration.url.uri); + object_storage = configuration->createObjectStorage(context_, /* is_readonly */true); + FormatFactory::instance().checkFormatName(configuration->format); + configuration->check(context_); + + ColumnsDescription columns{columns_}; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context_); + configuration->check(context_); StorageInMemoryMetadata storage_metadata; - if (columns_.empty()) - { - ColumnsDescription columns; - if (configuration.format == "auto") - std::tie(columns, configuration.format) = StorageS3::getTableStructureAndFormatFromData(configuration, format_settings, context_); - else - columns = StorageS3::getTableStructureFromData(configuration, format_settings, context_); - storage_metadata.setColumns(columns); - } - else - { - if (configuration.format == "auto") - configuration.format = StorageS3::getTableStructureAndFormatFromData(configuration, format_settings, context_).second; - storage_metadata.setColumns(columns_); - } - + storage_metadata.setColumns(columns); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setInMemoryMetadata(storage_metadata); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); task = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); @@ -235,7 +221,7 @@ void StorageS3Queue::drop() bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context_, format_settings); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context_, format_settings); } class ReadFromS3Queue : public SourceStepWithFilter @@ -364,44 +350,21 @@ std::shared_ptr StorageS3Queue::createSource( size_t max_block_size, ContextPtr local_context) { - auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); - - auto internal_source = std::make_unique( - info, - configuration.format, + auto internal_source = std::make_unique( getName(), - local_context, + object_storage, + configuration, + info, format_settings, + local_context, max_block_size, - configuration_snapshot.request_settings, - configuration_snapshot.compression_method, - configuration_snapshot.client, - configuration_snapshot.url.bucket, - configuration_snapshot.url.version_id, - configuration_snapshot.url.uri.getHost() + std::to_string(configuration_snapshot.url.uri.getPort()), file_iterator, local_context->getSettingsRef().max_download_threads, false); - auto file_deleter = [this, bucket = configuration_snapshot.url.bucket, client = configuration_snapshot.client, blob_storage_log = BlobStorageLogWriter::create()](const std::string & path) mutable + auto file_deleter = [=, this](const std::string & path) mutable { - S3::DeleteObjectRequest request; - request.WithKey(path).WithBucket(bucket); - auto outcome = client->DeleteObject(request); - if (blob_storage_log) - blob_storage_log->addEvent( - BlobStorageLogElement::EventType::Delete, - bucket, path, {}, 0, outcome.IsSuccess() ? nullptr : &outcome.GetError()); - - if (!outcome.IsSuccess()) - { - const auto & err = outcome.GetError(); - LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); - } - else - { - LOG_TRACE(log, "Object with path {} was removed from S3", path); - } + object_storage->removeObject(StoredObject(path)); }; auto s3_queue_log = s3queue_settings->s3queue_enable_logging_to_s3queue_log ? local_context->getS3QueueLog() : nullptr; return std::make_shared( @@ -495,7 +458,6 @@ bool StorageS3Queue::streamToViews() auto s3queue_context = Context::createCopy(getContext()); s3queue_context->makeQueryContext(); - auto query_configuration = updateConfigurationAndGetCopy(s3queue_context); // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns @@ -530,12 +492,6 @@ bool StorageS3Queue::streamToViews() return rows > 0; } -StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) -{ - configuration.update(local_context); - return configuration; -} - zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const { return getContext()->getZooKeeper(); @@ -555,7 +511,7 @@ void StorageS3Queue::createOrCheckMetadata(const StorageInMemoryMetadata & stora } else { - std::string metadata = S3QueueTableMetadata(configuration, *s3queue_settings, storage_metadata).toString(); + std::string metadata = S3QueueTableMetadata(*configuration, *s3queue_settings, storage_metadata).toString(); requests.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); requests.emplace_back(zkutil::makeCreateRequest(zk_path / "processed", "", zkutil::CreateMode::Persistent)); requests.emplace_back(zkutil::makeCreateRequest(zk_path / "failed", "", zkutil::CreateMode::Persistent)); @@ -597,7 +553,7 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata"); auto metadata_from_zk = S3QueueTableMetadata::parse(metadata_str); - S3QueueTableMetadata old_metadata(configuration, *s3queue_settings, storage_metadata); + S3QueueTableMetadata old_metadata(*configuration, *s3queue_settings, storage_metadata); old_metadata.checkEquals(metadata_from_zk); auto columns_from_zk = ColumnsDescription::parse(metadata_from_zk.columns); @@ -615,14 +571,9 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate) { - auto glob_iterator = std::make_unique( - *configuration.client, - configuration.url, - predicate, - getVirtualsList(), - local_context, - /* read_keys */ nullptr, - configuration.request_settings); + auto settings = configuration->getQuerySettings(local_context); + auto glob_iterator = std::make_unique( + object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match); return std::make_shared( files_metadata, std::move(glob_iterator), s3queue_settings->s3queue_current_shard_num, shutdown_called, log); @@ -638,7 +589,8 @@ void registerStorageS3Queue(StorageFactory & factory) if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getContext(), false); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index fce6736aa07..83b7bc6667b 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -21,11 +21,11 @@ class S3QueueFilesMetadata; class StorageS3Queue : public IStorage, WithContext { public: - using Configuration = typename StorageS3::Configuration; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; StorageS3Queue( std::unique_ptr s3queue_settings_, - const Configuration & configuration_, + ConfigurationPtr configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -47,7 +47,7 @@ public: size_t max_block_size, size_t num_streams) override; - const auto & getFormatName() const { return configuration.format; } + const auto & getFormatName() const { return configuration->format; } const fs::path & getZooKeeperPath() const { return zk_path; } @@ -62,7 +62,8 @@ private: const S3QueueAction after_processing; std::shared_ptr files_metadata; - Configuration configuration; + ConfigurationPtr configuration; + ObjectStoragePtr object_storage; const std::optional format_settings; @@ -97,7 +98,6 @@ private: void createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata); void checkTableStructure(const String & zookeeper_prefix, const StorageInMemoryMetadata & storage_metadata); - Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp deleted file mode 100644 index 365f93cc324..00000000000 --- a/src/Storages/StorageAzureBlob.cpp +++ /dev/null @@ -1,1643 +0,0 @@ -#include - -#if USE_AZURE_BLOB_STORAGE -#include -#include -#include -#include -#include -#include - -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include - -#include - -namespace fs = std::filesystem; - -using namespace Azure::Storage::Blobs; - -namespace CurrentMetrics -{ - extern const Metric ObjectStorageAzureThreads; - extern const Metric ObjectStorageAzureThreadsActive; - extern const Metric ObjectStorageAzureThreadsScheduled; -} - -namespace ProfileEvents -{ - extern const Event EngineFileLikeReadFiles; -} - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_COMPILE_REGEXP; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int CANNOT_DETECT_FORMAT; - extern const int LOGICAL_ERROR; - extern const int NOT_IMPLEMENTED; -} - -namespace -{ - -const std::unordered_set required_configuration_keys = { - "blob_path", - "container", -}; - -const std::unordered_set optional_configuration_keys = { - "format", - "compression", - "structure", - "compression_method", - "account_name", - "account_key", - "connection_string", - "storage_account_url", -}; - -bool isConnectionString(const std::string & candidate) -{ - return !candidate.starts_with("http"); -} - -} - -void StorageAzureBlob::processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection) -{ - validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); - - if (collection.has("connection_string")) - { - configuration.connection_url = collection.get("connection_string"); - configuration.is_connection_string = true; - } - - if (collection.has("storage_account_url")) - { - configuration.connection_url = collection.get("storage_account_url"); - configuration.is_connection_string = false; - } - - configuration.container = collection.get("container"); - configuration.blob_path = collection.get("blob_path"); - - if (collection.has("account_name")) - configuration.account_name = collection.get("account_name"); - - if (collection.has("account_key")) - configuration.account_key = collection.get("account_key"); - - configuration.structure = collection.getOrDefault("structure", "auto"); - configuration.format = collection.getOrDefault("format", configuration.format); - configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); -} - - -StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, const ContextPtr & local_context) -{ - StorageAzureBlob::Configuration configuration; - - /// Supported signatures: - /// - /// AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) - /// - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - { - processNamedCollectionResult(configuration, *named_collection); - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.blob_path).value_or("auto"); - - return configuration; - } - - if (engine_args.size() < 3 || engine_args.size() > 7) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage AzureBlobStorage requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression])"); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); - - std::unordered_map engine_args_to_idx; - - configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); - configuration.is_connection_string = isConnectionString(configuration.connection_url); - - configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); - configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); - - auto is_format_arg = [] (const std::string & s) -> bool - { - return s == "auto" || FormatFactory::instance().exists(s); - }; - - if (engine_args.size() == 4) - { - //'c1 UInt64, c2 UInt64 - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format or account name specified without account key"); - } - } - else if (engine_args.size() == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - } - } - else if (engine_args.size() == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); - } - else - { - configuration.account_name = fourth_arg; - - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - } - } - else if (engine_args.size() == 7) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - } - } - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.blob_path).value_or("auto"); - - return configuration; -} - - -AzureObjectStorage::SettingsPtr StorageAzureBlob::createSettings(const ContextPtr & local_context) -{ - const auto & context_settings = local_context->getSettingsRef(); - auto settings_ptr = std::make_unique(); - settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; - settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; - settings_ptr->strict_upload_part_size = context_settings.azure_strict_upload_part_size; - settings_ptr->max_upload_part_size = context_settings.azure_max_upload_part_size; - settings_ptr->max_blocks_in_multipart_upload = context_settings.azure_max_blocks_in_multipart_upload; - settings_ptr->min_upload_part_size = context_settings.azure_min_upload_part_size; - settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); - - return settings_ptr; -} - -void registerStorageAzureBlob(StorageFactory & factory) -{ - factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) - { - auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - - auto configuration = StorageAzureBlob::getConfiguration(engine_args, args.getLocalContext()); - auto client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); - // Use format settings from global server context + settings from - // the SETTINGS clause of the create query. Settings from current - // session and user are ignored. - std::optional format_settings; - if (args.storage_def->settings) - { - FormatFactorySettings user_format_settings; - - // Apply changed settings from global context, but ignore the - // unknown ones, because we only have the format settings here. - const auto & changes = args.getContext()->getSettingsRef().changes(); - for (const auto & change : changes) - { - if (user_format_settings.has(change.name)) - user_format_settings.set(change.name, change.value); - } - - // Apply changes from SETTINGS clause, with validation. - user_format_settings.applyChanges(args.storage_def->settings->changes); - format_settings = getFormatSettings(args.getContext(), user_format_settings); - } - else - { - format_settings = getFormatSettings(args.getContext()); - } - - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - - auto settings = StorageAzureBlob::createSettings(args.getContext()); - - return std::make_shared( - configuration, - std::make_unique("AzureBlobStorage", std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()), - args.getContext(), - args.table_id, - args.columns, - args.constraints, - args.comment, - format_settings, - /* distributed_processing */ false, - partition_by); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::AZURE, - }); -} - -static bool containerExists(std::unique_ptr &blob_service_client, std::string container_name) -{ - Azure::Storage::Blobs::ListBlobContainersOptions options; - options.Prefix = container_name; - options.PageSizeHint = 1; - - auto containers_list_response = blob_service_client->ListBlobContainers(options); - auto containers_list = containers_list_response.BlobContainers; - - for (const auto & container : containers_list) - { - if (container_name == container.Name) - return true; - } - return false; -} - -AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration, bool is_read_only, bool attempt_to_create_container) -{ - AzureClientPtr result; - - if (configuration.is_connection_string) - { - std::shared_ptr managed_identity_credential = std::make_shared(); - std::unique_ptr blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(configuration.connection_url)); - result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); - - if (attempt_to_create_container) - { - bool container_exists = containerExists(blob_service_client,configuration.container); - if (!container_exists) - { - if (is_read_only) - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage container does not exist '{}'", - configuration.container); - - try - { - result->CreateIfNotExists(); - } - catch (const Azure::Storage::StorageException & e) - { - if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict - && e.ReasonPhrase == "The specified container already exists.")) - { - throw; - } - } - } - } - } - else - { - std::shared_ptr storage_shared_key_credential; - if (configuration.account_name.has_value() && configuration.account_key.has_value()) - { - storage_shared_key_credential - = std::make_shared(*configuration.account_name, *configuration.account_key); - } - - std::unique_ptr blob_service_client; - size_t pos = configuration.connection_url.find('?'); - std::shared_ptr managed_identity_credential; - if (storage_shared_key_credential) - { - blob_service_client = std::make_unique(configuration.connection_url, storage_shared_key_credential); - } - else - { - /// If conneciton_url does not have '?', then its not SAS - if (pos == std::string::npos) - { - auto workload_identity_credential = std::make_shared(); - blob_service_client = std::make_unique(configuration.connection_url, workload_identity_credential); - } - else - { - managed_identity_credential = std::make_shared(); - blob_service_client = std::make_unique(configuration.connection_url, managed_identity_credential); - } - } - - std::string final_url; - if (pos != std::string::npos) - { - auto url_without_sas = configuration.connection_url.substr(0, pos); - final_url = url_without_sas + (url_without_sas.back() == '/' ? "" : "/") + configuration.container - + configuration.connection_url.substr(pos); - } - else - final_url - = configuration.connection_url + (configuration.connection_url.back() == '/' ? "" : "/") + configuration.container; - - if (!attempt_to_create_container) - { - if (storage_shared_key_credential) - return std::make_unique(final_url, storage_shared_key_credential); - else - return std::make_unique(final_url, managed_identity_credential); - } - - bool container_exists = containerExists(blob_service_client,configuration.container); - if (container_exists) - { - if (storage_shared_key_credential) - result = std::make_unique(final_url, storage_shared_key_credential); - else - { - /// If conneciton_url does not have '?', then its not SAS - if (pos == std::string::npos) - { - auto workload_identity_credential = std::make_shared(); - result = std::make_unique(final_url, workload_identity_credential); - } - else - result = std::make_unique(final_url, managed_identity_credential); - } - } - else - { - if (is_read_only) - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage container does not exist '{}'", - configuration.container); - try - { - result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); - } - catch (const Azure::Storage::StorageException & e) - { - if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict - && e.ReasonPhrase == "The specified container already exists.") - { - if (storage_shared_key_credential) - result = std::make_unique(final_url, storage_shared_key_credential); - else - { - /// If conneciton_url does not have '?', then its not SAS - if (pos == std::string::npos) - { - auto workload_identity_credential = std::make_shared(); - result = std::make_unique(final_url, workload_identity_credential); - } - else - result = std::make_unique(final_url, managed_identity_credential); - } - } - else - { - throw; - } - } - } - } - - return result; -} - -Poco::URI StorageAzureBlob::Configuration::getConnectionURL() const -{ - if (!is_connection_string) - return Poco::URI(connection_url); - - auto parsed_connection_string = Azure::Storage::_internal::ParseConnectionString(connection_url); - return Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); -} - -bool StorageAzureBlob::Configuration::withGlobsIgnorePartitionWildcard() const -{ - if (!withPartitionWildcard()) - return withGlobs(); - - return PartitionedSink::replaceWildcards(getPath(), "").find_first_of("*?{") != std::string::npos; -} - -StorageAzureBlob::StorageAzureBlob( - const Configuration & configuration_, - std::unique_ptr && object_storage_, - const ContextPtr & context, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_, - bool distributed_processing_, - ASTPtr partition_by_) - : IStorage(table_id_) - , name("AzureBlobStorage") - , configuration(configuration_) - , object_storage(std::move(object_storage_)) - , distributed_processing(distributed_processing_) - , format_settings(format_settings_) - , partition_by(partition_by_) -{ - if (configuration.format != "auto") - FormatFactory::instance().checkFormatName(configuration.format); - context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.getConnectionURL()); - - StorageInMemoryMetadata storage_metadata; - if (columns_.empty()) - { - ColumnsDescription columns; - if (configuration.format == "auto") - std::tie(columns, configuration.format) = getTableStructureAndFormatFromData(object_storage.get(), configuration, format_settings, context); - else - columns = getTableStructureFromData(object_storage.get(), configuration, format_settings, context); - storage_metadata.setColumns(columns); - } - else - { - if (configuration.format == "auto") - configuration.format = getTableStructureAndFormatFromData(object_storage.get(), configuration, format_settings, context).second; - - /// We don't allow special columns in File storage. - if (!columns_.hasOnlyOrdinary()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine AzureBlobStorage doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL"); - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); - - StoredObjects objects; - for (const auto & key : configuration.blobs_paths) - objects.emplace_back(key); -} - -void StorageAzureBlob::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) -{ - if (configuration.withGlobs()) - { - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage key '{}' contains globs, so the table is in readonly mode", - configuration.blob_path); - } - - StoredObjects objects; - for (const auto & key : configuration.blobs_paths) - objects.emplace_back(key); - - object_storage->removeObjectsIfExist(objects); -} - -namespace -{ - -class StorageAzureBlobSink : public SinkToStorage -{ -public: - StorageAzureBlobSink( - const String & format, - const Block & sample_block_, - const ContextPtr & context, - std::optional format_settings_, - const CompressionMethod compression_method, - AzureObjectStorage * object_storage, - const String & blob_path) - : SinkToStorage(sample_block_) - , sample_block(sample_block_) - , format_settings(format_settings_) - { - StoredObject object(blob_path); - const auto & settings = context->getSettingsRef(); - write_buf = wrapWriteBufferWithCompressionMethod( - object_storage->writeObject(object, WriteMode::Rewrite), - compression_method, - static_cast(settings.output_format_compression_level), - static_cast(settings.output_format_compression_zstd_window_log)); - writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); - } - - String getName() const override { return "StorageAzureBlobSink"; } - - void consume(Chunk chunk) override - { - std::lock_guard lock(cancel_mutex); - if (cancelled) - return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); - } - - void onCancel() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - cancelled = true; - } - - void onException(std::exception_ptr exception) override - { - std::lock_guard lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization - release(); - } - } - - void onFinish() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - } - -private: - void finalize() - { - if (!writer) - return; - - try - { - writer->finalize(); - writer->flush(); - write_buf->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - release(); - throw; - } - } - - void release() - { - writer.reset(); - write_buf->finalize(); - } - - Block sample_block; - std::optional format_settings; - std::unique_ptr write_buf; - OutputFormatPtr writer; - bool cancelled = false; - std::mutex cancel_mutex; -}; - -namespace -{ - std::optional checkAndGetNewFileOnInsertIfNeeded(const ContextPtr & context, AzureObjectStorage * object_storage, const String & path, size_t sequence_number) - { - if (context->getSettingsRef().azure_truncate_on_insert || !object_storage->exists(StoredObject(path))) - return std::nullopt; - - if (context->getSettingsRef().azure_create_new_file_on_insert) - { - auto pos = path.find_first_of('.'); - String new_path; - do - { - new_path = path.substr(0, pos) + "." + std::to_string(sequence_number) + (pos == std::string::npos ? "" : path.substr(pos)); - ++sequence_number; - } - while (object_storage->exists(StoredObject(new_path))); - - return new_path; - } - - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Object with key {} already exists. " - "If you want to overwrite it, enable setting azure_truncate_on_insert, if you " - "want to create a new file on each insert, enable setting azure_create_new_file_on_insert", - path); - } -} - -class PartitionedStorageAzureBlobSink : public PartitionedSink, WithContext -{ -public: - PartitionedStorageAzureBlobSink( - const ASTPtr & partition_by, - const String & format_, - const Block & sample_block_, - const ContextPtr & context_, - std::optional format_settings_, - const CompressionMethod compression_method_, - AzureObjectStorage * object_storage_, - const String & blob_) - : PartitionedSink(partition_by, context_, sample_block_), WithContext(context_) - , format(format_) - , sample_block(sample_block_) - , compression_method(compression_method_) - , object_storage(object_storage_) - , blob(blob_) - , format_settings(format_settings_) - { - } - - SinkPtr createSinkForPartition(const String & partition_id) override - { - auto partition_key = replaceWildcards(blob, partition_id); - validateKey(partition_key); - if (auto new_path = checkAndGetNewFileOnInsertIfNeeded(getContext(), object_storage, partition_key, 1)) - partition_key = *new_path; - - return std::make_shared( - format, - sample_block, - getContext(), - format_settings, - compression_method, - object_storage, - partition_key - ); - } - -private: - const String format; - const Block sample_block; - const CompressionMethod compression_method; - AzureObjectStorage * object_storage; - const String blob; - const std::optional format_settings; - - ExpressionActionsPtr partition_by_expr; - - static void validateKey(const String & str) - { - validatePartitionKey(str, true); - } -}; - -} - -class ReadFromAzureBlob : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromAzureBlob"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - void applyFilters(ActionDAGNodes added_filter_nodes) override; - - ReadFromAzureBlob( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::shared_ptr storage_, - ReadFromFormatInfo info_, - const bool need_only_count_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) - , storage(std::move(storage_)) - , info(std::move(info_)) - , need_only_count(need_only_count_) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - { - } - -private: - std::shared_ptr storage; - ReadFromFormatInfo info; - const bool need_only_count; - - size_t max_block_size; - const size_t num_streams; - - std::shared_ptr iterator_wrapper; - - void createIterator(const ActionsDAG::Node * predicate); -}; - -void ReadFromAzureBlob::applyFilters(ActionDAGNodes added_filter_nodes) -{ - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - createIterator(predicate); -} - -void StorageAzureBlob::read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr local_context, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t num_streams) -{ - if (partition_by && configuration.withPartitionWildcard()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned Azure storage is not implemented yet"); - - auto this_ptr = std::static_pointer_cast(shared_from_this()); - - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && local_context->getSettingsRef().optimize_count_from_files; - - auto reading = std::make_unique( - column_names, - query_info, - storage_snapshot, - local_context, - read_from_format_info.source_header, - std::move(this_ptr), - std::move(read_from_format_info), - need_only_count, - max_block_size, - num_streams); - - query_plan.addStep(std::move(reading)); -} - -void ReadFromAzureBlob::createIterator(const ActionsDAG::Node * predicate) -{ - if (iterator_wrapper) - return; - - const auto & configuration = storage->configuration; - - if (storage->distributed_processing) - { - iterator_wrapper = std::make_shared(context, - context->getReadTaskCallback()); - } - else if (configuration.withGlobs()) - { - /// Iterate through disclosed globs and make a source for each file - iterator_wrapper = std::make_shared( - storage->object_storage.get(), configuration.container, configuration.blob_path, - predicate, storage->getVirtualsList(), context, nullptr, context->getFileProgressCallback()); - } - else - { - iterator_wrapper = std::make_shared( - storage->object_storage.get(), configuration.container, configuration.blobs_paths, - predicate, storage->getVirtualsList(), context, nullptr, context->getFileProgressCallback()); - } -} - -void ReadFromAzureBlob::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - createIterator(nullptr); - - const auto & configuration = storage->configuration; - Pipes pipes; - - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - info, - configuration.format, - getName(), - context, - storage->format_settings, - max_block_size, - configuration.compression_method, - storage->object_storage.get(), - configuration.container, - configuration.connection_url, - iterator_wrapper, - need_only_count)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); -} - -SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) -{ - if (configuration.withGlobsIgnorePartitionWildcard()) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); - - auto path = configuration.blobs_paths.front(); - auto sample_block = metadata_snapshot->getSampleBlock(); - auto chosen_compression_method = chooseCompressionMethod(path, configuration.compression_method); - auto insert_query = std::dynamic_pointer_cast(query); - - auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && configuration.withPartitionWildcard(); - - if (is_partitioned_implementation) - { - return std::make_shared( - partition_by_ast, - configuration.format, - sample_block, - local_context, - format_settings, - chosen_compression_method, - object_storage.get(), - path); - } - else - { - if (auto new_path = checkAndGetNewFileOnInsertIfNeeded(local_context, object_storage.get(), path, configuration.blobs_paths.size())) - { - configuration.blobs_paths.push_back(*new_path); - path = *new_path; - } - - return std::make_shared( - configuration.format, - sample_block, - local_context, - format_settings, - chosen_compression_method, - object_storage.get(), - path); - } -} - -bool StorageAzureBlob::supportsPartitionBy() const -{ - return true; -} - -bool StorageAzureBlob::supportsSubsetOfColumns(const ContextPtr & context) const -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context, format_settings); -} - -bool StorageAzureBlob::prefersLargeBlocks() const -{ - return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration.format); -} - -bool StorageAzureBlob::parallelizeOutputAfterReading(ContextPtr context) const -{ - return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration.format, context); -} - -StorageAzureBlobSource::GlobIterator::GlobIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - String blob_path_with_globs_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - const ContextPtr & context_, - RelativePathsWithMetadata * outer_blobs_, - std::function file_progress_callback_) - : IIterator(context_) - , object_storage(object_storage_) - , container(container_) - , blob_path_with_globs(blob_path_with_globs_) - , virtual_columns(virtual_columns_) - , outer_blobs(outer_blobs_) - , file_progress_callback(file_progress_callback_) -{ - - const String key_prefix = blob_path_with_globs.substr(0, blob_path_with_globs.find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == blob_path_with_globs.size()) - { - auto object_metadata = object_storage->getObjectMetadata(blob_path_with_globs); - blobs_with_metadata.emplace_back( - blob_path_with_globs, - object_metadata); - if (outer_blobs) - outer_blobs->emplace_back(blobs_with_metadata.back()); - if (file_progress_callback) - file_progress_callback(FileProgress(0, object_metadata.size_bytes)); - is_finished = true; - return; - } - - object_storage_iterator = object_storage->iterate(key_prefix); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(blob_path_with_globs)); - - if (!matcher->ok()) - throw Exception( - ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", blob_path_with_globs, matcher->error()); - - recursive = blob_path_with_globs == "/**" ? true : false; - - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); -} - -RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() -{ - std::lock_guard lock(next_mutex); - - if (is_finished && index >= blobs_with_metadata.size()) - { - return {}; - } - - bool need_new_batch = blobs_with_metadata.empty() || index >= blobs_with_metadata.size(); - - if (need_new_batch) - { - RelativePathsWithMetadata new_batch; - while (new_batch.empty()) - { - auto result = object_storage_iterator->getCurrrentBatchAndScheduleNext(); - if (result.has_value()) - { - new_batch = result.value(); - } - else - { - is_finished = true; - return {}; - } - - for (auto it = new_batch.begin(); it != new_batch.end();) - { - if (!recursive && !re2::RE2::FullMatch(it->relative_path, *matcher)) - it = new_batch.erase(it); - else - ++it; - } - } - - index = 0; - - if (filter_dag) - { - std::vector paths; - paths.reserve(new_batch.size()); - for (auto & path_with_metadata : new_batch) - paths.push_back(fs::path(container) / path_with_metadata.relative_path); - - VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); - } - - if (outer_blobs) - outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); - - blobs_with_metadata = std::move(new_batch); - if (file_progress_callback) - { - for (const auto & [relative_path, info] : blobs_with_metadata) - { - file_progress_callback(FileProgress(0, info.size_bytes)); - } - } - } - - size_t current_index = index++; - if (current_index >= blobs_with_metadata.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Index out of bound for blob metadata"); - return blobs_with_metadata[current_index]; -} - -StorageAzureBlobSource::KeysIterator::KeysIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - const Strings & keys_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - const ContextPtr & context_, - RelativePathsWithMetadata * outer_blobs, - std::function file_progress_callback) - : IIterator(context_) - , object_storage(object_storage_) - , container(container_) - , virtual_columns(virtual_columns_) -{ - Strings all_keys = keys_; - - ASTPtr filter_ast; - if (!all_keys.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - - if (filter_dag) - { - Strings paths; - paths.reserve(all_keys.size()); - for (const auto & key : all_keys) - paths.push_back(fs::path(container) / key); - - VirtualColumnUtils::filterByPathOrFile(all_keys, paths, filter_dag, virtual_columns, getContext()); - } - - for (auto && key : all_keys) - { - ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); - if (file_progress_callback) - file_progress_callback(FileProgress(0, object_metadata.size_bytes)); - keys.emplace_back(key, object_metadata); - } - - if (outer_blobs) - *outer_blobs = keys; -} - -RelativePathWithMetadata StorageAzureBlobSource::KeysIterator::next() -{ - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= keys.size()) - return {}; - - return keys[current_index]; -} - -Chunk StorageAzureBlobSource::generate() -{ - while (true) - { - if (isCancelled() || !reader) - { - if (reader) - reader->cancel(); - break; - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - total_rows_in_file += num_rows; - size_t chunk_size = 0; - if (const auto * input_format = reader.getInputFormat()) - chunk_size = input_format->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, - requested_virtual_columns, - fs::path(container) / reader.getRelativePath(), - reader.getRelativePathWithMetadata().metadata.size_bytes); - return chunk; - } - - if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(reader.getRelativePath(), total_rows_in_file); - - total_rows_in_file = 0; - - assert(reader_future.valid()); - reader = reader_future.get(); - - if (!reader) - break; - - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - create_reader_pool.wait(); - reader_future = createReaderAsync(); - } - - return {}; -} - -void StorageAzureBlobSource::addNumRowsToCache(const String & path, size_t num_rows) -{ - String source = fs::path(connection_url) / container / path; - auto cache_key = getKeyForSchemaCache(source, format, format_settings, getContext()); - StorageAzureBlob::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); -} - -std::optional StorageAzureBlobSource::tryGetNumRowsFromCache(const DB::RelativePathWithMetadata & path_with_metadata) -{ - String source = fs::path(connection_url) / container / path_with_metadata.relative_path; - auto cache_key = getKeyForSchemaCache(source, format, format_settings, getContext()); - auto get_last_mod_time = [&]() -> std::optional - { - auto last_mod = path_with_metadata.metadata.last_modified; - if (last_mod) - return last_mod->epochTime(); - return std::nullopt; - }; - - return StorageAzureBlob::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); -} - -StorageAzureBlobSource::StorageAzureBlobSource( - const ReadFromFormatInfo & info, - const String & format_, - String name_, - const ContextPtr & context_, - std::optional format_settings_, - UInt64 max_block_size_, - String compression_hint_, - AzureObjectStorage * object_storage_, - const String & container_, - const String & connection_url_, - std::shared_ptr file_iterator_, - bool need_only_count_) - :ISource(info.source_header, false) - , WithContext(context_) - , requested_columns(info.requested_columns) - , requested_virtual_columns(info.requested_virtual_columns) - , format(format_) - , name(std::move(name_)) - , sample_block(info.format_header) - , format_settings(format_settings_) - , columns_desc(info.columns_description) - , max_block_size(max_block_size_) - , compression_hint(compression_hint_) - , object_storage(std::move(object_storage_)) - , container(container_) - , connection_url(connection_url_) - , file_iterator(file_iterator_) - , need_only_count(need_only_count_) - , create_reader_pool(CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, CurrentMetrics::ObjectStorageAzureThreadsScheduled, 1) - , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(create_reader_pool, "AzureReader")) -{ - reader = createReader(); - if (reader) - reader_future = createReaderAsync(); -} - - -StorageAzureBlobSource::~StorageAzureBlobSource() -{ - create_reader_pool.wait(); -} - -String StorageAzureBlobSource::getName() const -{ - return name; -} - -StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() -{ - auto path_with_metadata = file_iterator->next(); - if (path_with_metadata.relative_path.empty()) - return {}; - - if (path_with_metadata.metadata.size_bytes == 0) - path_with_metadata.metadata = object_storage->getObjectMetadata(path_with_metadata.relative_path); - - QueryPipelineBuilder builder; - std::shared_ptr source; - std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files - ? tryGetNumRowsFromCache(path_with_metadata) : std::nullopt; - if (num_rows_from_cache) - { - /// We should not return single chunk with all number of rows, - /// because there is a chance that this chunk will be materialized later - /// (it can cause memory problems even with default values in columns or when virtual columns are requested). - /// Instead, we use special ConstChunkGenerator that will generate chunks - /// with max_block_size rows until total number of rows is reached. - source = std::make_shared(sample_block, *num_rows_from_cache, max_block_size); - builder.init(Pipe(source)); - } - else - { - std::optional max_parsing_threads; - if (need_only_count) - max_parsing_threads = 1; - - auto compression_method = chooseCompressionMethod(path_with_metadata.relative_path, compression_hint); - read_buf = createAzureReadBuffer(path_with_metadata.relative_path, path_with_metadata.metadata.size_bytes); - auto input_format = FormatFactory::instance().getInput( - format, *read_buf, sample_block, getContext(), max_block_size, - format_settings, max_parsing_threads, std::nullopt, - /* is_remote_fs */ true, compression_method); - - if (need_only_count) - input_format->needOnlyCount(); - - builder.init(Pipe(input_format)); - - if (columns_desc.hasDefaults()) - { - builder.addSimpleTransform( - [&](const Block & header) - { return std::make_shared(header, columns_desc, *input_format, getContext()); }); - } - - source = input_format; - } - - /// Add ExtractColumnsTransform to extract requested columns/subcolumns - /// from chunk read by IInputFormat. - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, requested_columns); - }); - - auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - auto current_reader = std::make_unique(*pipeline); - - ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - - return ReaderHolder{path_with_metadata, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; -} - -std::future StorageAzureBlobSource::createReaderAsync() -{ - return create_reader_scheduler([this] { return createReader(); }, Priority{}); -} - -std::unique_ptr StorageAzureBlobSource::createAzureReadBuffer(const String & key, size_t object_size) -{ - auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); - read_settings.enable_filesystem_cache = false; - auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - const bool object_too_small = object_size <= 2 * download_buffer_size; - - // Create a read buffer that will prefetch the first ~1 MB of the file. - // When reading lots of tiny files, this prefetching almost doubles the throughput. - // For bigger files, parallel reading is more useful. - if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - LOG_TRACE(log, "Downloading object of size {} from Azure with initial prefetch", object_size); - return createAsyncAzureReadBuffer(key, read_settings, object_size); - } - - return object_storage->readObject(StoredObject(key), read_settings, {}, object_size); -} - -namespace -{ - class ReadBufferIterator : public IReadBufferIterator, WithContext - { - public: - ReadBufferIterator( - const std::shared_ptr & file_iterator_, - AzureObjectStorage * object_storage_, - std::optional format_, - const StorageAzureBlob::Configuration & configuration_, - const std::optional & format_settings_, - const RelativePathsWithMetadata & read_keys_, - const ContextPtr & context_) - : WithContext(context_) - , file_iterator(file_iterator_) - , object_storage(object_storage_) - , configuration(configuration_) - , format(std::move(format_)) - , format_settings(format_settings_) - , read_keys(read_keys_) - , prev_read_keys_size(read_keys_.size()) - { - } - - Data next() override - { - /// For default mode check cached columns for currently read keys on first iteration. - if (first) - { - /// If format is unknown we iterate through all currently read keys on first iteration and - /// try to determine format by file name. - if (!format) - { - for (const auto & key : read_keys) - { - if (auto format_from_path = FormatFactory::instance().tryGetFormatFromFileName(key.relative_path)) - { - format = format_from_path; - break; - } - } - } - - /// For default mode check cached columns for currently read keys on first iteration. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) - return {nullptr, cached_columns, format}; - } - } - - current_path_with_metadata = file_iterator->next(); - - if (current_path_with_metadata.relative_path.empty()) - { - if (first) - { - if (format) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The table structure cannot be extracted from a {} format file, because there are no files with provided path " - "in AzureBlobStorage. You can specify table structure manually", *format); - - throw Exception( - ErrorCodes::CANNOT_DETECT_FORMAT, - "The data format cannot be detected by the contents of the files, because there are no files with provided path " - "in AzureBlobStorage. You can specify table structure manually"); - } - - return {nullptr, std::nullopt, format}; - } - - first = false; - - /// AzureBlobStorage file iterator could get new keys after new iteration. - if (read_keys.size() > prev_read_keys_size) - { - /// If format is unknown we can try to determine it by new file names. - if (!format) - { - for (auto it = read_keys.begin() + prev_read_keys_size; it != read_keys.end(); ++it) - { - if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it).relative_path)) - { - format = format_from_file_name; - break; - } - } - } - /// Check new files in schema cache if schema inference mode is default. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); - if (columns_from_cache) - return {nullptr, columns_from_cache, format}; - } - - prev_read_keys_size = read_keys.size(); - } - - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - RelativePathsWithMetadata paths = {current_path_with_metadata}; - if (auto columns_from_cache = tryGetColumnsFromCache(paths.begin(), paths.end())) - return {nullptr, columns_from_cache, format}; - } - - first = false; - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - return {wrapReadBufferWithCompressionMethod( - object_storage->readObject(StoredObject(current_path_with_metadata.relative_path), getContext()->getReadSettings(), {}, current_path_with_metadata.metadata.size_bytes), - chooseCompressionMethod(current_path_with_metadata.relative_path, configuration.compression_method), - zstd_window_log_max), std::nullopt, format}; - } - - void setNumRowsToLastFile(size_t num_rows) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure) - return; - - String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path; - auto key = getKeyForSchemaCache(source, *format, format_settings, getContext()); - StorageAzureBlob::getSchemaCache(getContext()).addNumRows(key, num_rows); - } - - void setSchemaToLastFile(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) - return; - - String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path; - auto key = getKeyForSchemaCache(source, *format, format_settings, getContext()); - StorageAzureBlob::getSchemaCache(getContext()).addColumns(key, columns); - } - - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) - return; - - auto host_and_bucket = configuration.connection_url + '/' + configuration.container; - Strings sources; - sources.reserve(read_keys.size()); - std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket + '/' + elem.relative_path; }); - auto cache_keys = getKeysForSchemaCache(sources, *format, format_settings, getContext()); - StorageAzureBlob::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); - } - - void setFormatName(const String & format_name) override - { - format = format_name; - } - - String getLastFileName() const override { return current_path_with_metadata.relative_path; } - - bool supportsLastReadBufferRecreation() const override { return true; } - - std::unique_ptr recreateLastReadBuffer() override - { - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - return wrapReadBufferWithCompressionMethod( - object_storage->readObject(StoredObject(current_path_with_metadata.relative_path), getContext()->getReadSettings(), {}, current_path_with_metadata.metadata.size_bytes), - chooseCompressionMethod(current_path_with_metadata.relative_path, configuration.compression_method), - zstd_window_log_max); - } - - private: - std::optional tryGetColumnsFromCache(const RelativePathsWithMetadata::const_iterator & begin, const RelativePathsWithMetadata::const_iterator & end) - { - auto context = getContext(); - if (!context->getSettingsRef().schema_inference_use_cache_for_azure) - return std::nullopt; - - auto & schema_cache = StorageAzureBlob::getSchemaCache(context); - for (auto it = begin; it < end; ++it) - { - auto get_last_mod_time = [&] -> std::optional - { - if (it->metadata.last_modified) - return it->metadata.last_modified->epochTime(); - return std::nullopt; - }; - - auto host_and_bucket = configuration.connection_url + '/' + configuration.container; - String source = host_and_bucket + '/' + it->relative_path; - if (format) - { - auto cache_key = getKeyForSchemaCache(source, *format, format_settings, context); - if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) - return columns; - } - else - { - /// If format is unknown, we can iterate through all possible input formats - /// and check if we have an entry with this format and this file in schema cache. - /// If we have such entry for some format, we can use this format to read the file. - for (const auto & format_name : FormatFactory::instance().getAllInputFormats()) - { - auto cache_key = getKeyForSchemaCache(source, format_name, format_settings, context); - if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) - { - /// Now format is known. It should be the same for all files. - format = format_name; - return columns; - } - } - } - } - - return std::nullopt; - } - - std::shared_ptr file_iterator; - AzureObjectStorage * object_storage; - const StorageAzureBlob::Configuration & configuration; - std::optional format; - const std::optional & format_settings; - const RelativePathsWithMetadata & read_keys; - size_t prev_read_keys_size; - RelativePathWithMetadata current_path_with_metadata; - bool first = true; - }; -} - -std::pair StorageAzureBlob::getTableStructureAndFormatFromDataImpl( - std::optional format, - AzureObjectStorage * object_storage, - const Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx) -{ - RelativePathsWithMetadata read_keys; - std::shared_ptr file_iterator; - if (configuration.withGlobs()) - { - file_iterator = std::make_shared( - object_storage, configuration.container, configuration.blob_path, nullptr, NamesAndTypesList{}, ctx, &read_keys); - } - else - { - file_iterator = std::make_shared( - object_storage, configuration.container, configuration.blobs_paths, nullptr, NamesAndTypesList{}, ctx, &read_keys); - } - - ReadBufferIterator read_buffer_iterator(file_iterator, object_storage, format, configuration, format_settings, read_keys, ctx); - if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, ctx), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, ctx); -} - -std::pair StorageAzureBlob::getTableStructureAndFormatFromData( - DB::AzureObjectStorage * object_storage, - const DB::StorageAzureBlob::Configuration & configuration, - const std::optional & format_settings, - const DB::ContextPtr & ctx) -{ - return getTableStructureAndFormatFromDataImpl(std::nullopt, object_storage, configuration, format_settings, ctx); -} - -ColumnsDescription StorageAzureBlob::getTableStructureFromData( - DB::AzureObjectStorage * object_storage, - const DB::StorageAzureBlob::Configuration & configuration, - const std::optional & format_settings, - const DB::ContextPtr & ctx) -{ - return getTableStructureAndFormatFromDataImpl(configuration.format, object_storage, configuration, format_settings, ctx).first; -} - -SchemaCache & StorageAzureBlob::getSchemaCache(const ContextPtr & ctx) -{ - static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_azure", DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; -} - - -std::unique_ptr StorageAzureBlobSource::createAsyncAzureReadBuffer( - const String & key, const ReadSettings & read_settings, size_t object_size) -{ - auto modified_settings{read_settings}; - modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; - auto async_reader = object_storage->readObjects(StoredObjects{StoredObject{key, /* local_path */ "", object_size}}, modified_settings); - - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; -} - -} - -#endif diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h deleted file mode 100644 index 04003d28035..00000000000 --- a/src/Storages/StorageAzureBlob.h +++ /dev/null @@ -1,349 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class StorageAzureBlob : public IStorage -{ -public: - - using AzureClient = Azure::Storage::Blobs::BlobContainerClient; - using AzureClientPtr = std::unique_ptr; - - struct Configuration : public StatelessTableEngineConfiguration - { - Configuration() = default; - - String getPath() const { return blob_path; } - - bool update(const ContextPtr & context); - - bool withGlobs() const { return blob_path.find_first_of("*?{") != std::string::npos; } - - bool withPartitionWildcard() const - { - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - return blobs_paths.back().find(PARTITION_ID_WILDCARD) != String::npos; - } - - bool withGlobsIgnorePartitionWildcard() const; - - Poco::URI getConnectionURL() const; - - std::string connection_url; - bool is_connection_string; - - std::optional account_name; - std::optional account_key; - - std::string container; - std::string blob_path; - std::vector blobs_paths; - }; - - StorageAzureBlob( - const Configuration & configuration_, - std::unique_ptr && object_storage_, - const ContextPtr & context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_, - bool distributed_processing_, - ASTPtr partition_by_); - - static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, const ContextPtr & local_context); - static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only, bool attempt_to_create_container = true); - - static AzureObjectStorage::SettingsPtr createSettings(const ContextPtr & local_context); - - static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection); - - String getName() const override - { - return name; - } - - void read( - QueryPlan & query_plan, - const Names &, - const StorageSnapshotPtr &, - SelectQueryInfo &, - ContextPtr, - QueryProcessingStage::Enum, - size_t, - size_t) override; - - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr context, bool /*async_insert*/) override; - - void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; - - bool supportsPartitionBy() const override; - - bool supportsSubcolumns() const override { return true; } - - bool supportsDynamicSubcolumns() const override { return true; } - - bool supportsSubsetOfColumns(const ContextPtr & context) const; - - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } - - bool prefersLargeBlocks() const override; - - bool parallelizeOutputAfterReading(ContextPtr context) const override; - - static SchemaCache & getSchemaCache(const ContextPtr & ctx); - - static ColumnsDescription getTableStructureFromData( - AzureObjectStorage * object_storage, - const Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx); - - static std::pair getTableStructureAndFormatFromData( - AzureObjectStorage * object_storage, - const Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx); - -private: - static std::pair getTableStructureAndFormatFromDataImpl( - std::optional format, - AzureObjectStorage * object_storage, - const Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx); - - friend class ReadFromAzureBlob; - - std::string name; - Configuration configuration; - std::unique_ptr object_storage; - - const bool distributed_processing; - std::optional format_settings; - ASTPtr partition_by; -}; - -class StorageAzureBlobSource : public ISource, WithContext -{ -public: - class IIterator : public WithContext - { - public: - explicit IIterator(const ContextPtr & context_):WithContext(context_) {} - virtual ~IIterator() = default; - virtual RelativePathWithMetadata next() = 0; - - RelativePathWithMetadata operator ()() { return next(); } - }; - - class GlobIterator : public IIterator - { - public: - GlobIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - String blob_path_with_globs_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - const ContextPtr & context_, - RelativePathsWithMetadata * outer_blobs_, - std::function file_progress_callback_ = {}); - - RelativePathWithMetadata next() override; - ~GlobIterator() override = default; - - private: - AzureObjectStorage * object_storage; - std::string container; - String blob_path_with_globs; - ActionsDAGPtr filter_dag; - NamesAndTypesList virtual_columns; - - size_t index = 0; - - RelativePathsWithMetadata blobs_with_metadata; - RelativePathsWithMetadata * outer_blobs; - ObjectStorageIteratorPtr object_storage_iterator; - bool recursive{false}; - - std::unique_ptr matcher; - - void createFilterAST(const String & any_key); - bool is_finished = false; - std::mutex next_mutex; - - std::function file_progress_callback; - }; - - class ReadIterator : public IIterator - { - public: - explicit ReadIterator(const ContextPtr & context_, - const ReadTaskCallback & callback_) - : IIterator(context_), callback(callback_) { } - RelativePathWithMetadata next() override - { - return {callback(), {}}; - } - - private: - ReadTaskCallback callback; - }; - - class KeysIterator : public IIterator - { - public: - KeysIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - const Strings & keys_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - const ContextPtr & context_, - RelativePathsWithMetadata * outer_blobs, - std::function file_progress_callback = {}); - - RelativePathWithMetadata next() override; - ~KeysIterator() override = default; - - private: - AzureObjectStorage * object_storage; - std::string container; - RelativePathsWithMetadata keys; - - ActionsDAGPtr filter_dag; - NamesAndTypesList virtual_columns; - - std::atomic index = 0; - }; - - StorageAzureBlobSource( - const ReadFromFormatInfo & info, - const String & format_, - String name_, - const ContextPtr & context_, - std::optional format_settings_, - UInt64 max_block_size_, - String compression_hint_, - AzureObjectStorage * object_storage_, - const String & container_, - const String & connection_url_, - std::shared_ptr file_iterator_, - bool need_only_count_); - ~StorageAzureBlobSource() override; - - Chunk generate() override; - - String getName() const override; - -private: - void addNumRowsToCache(const String & path, size_t num_rows); - std::optional tryGetNumRowsFromCache(const RelativePathWithMetadata & path_with_metadata); - - NamesAndTypesList requested_columns; - NamesAndTypesList requested_virtual_columns; - String format; - String name; - Block sample_block; - std::optional format_settings; - ColumnsDescription columns_desc; - UInt64 max_block_size; - String compression_hint; - AzureObjectStorage * object_storage; - String container; - String connection_url; - std::shared_ptr file_iterator; - bool need_only_count; - size_t total_rows_in_file = 0; - - struct ReaderHolder - { - public: - ReaderHolder( - RelativePathWithMetadata relative_path_with_metadata_, - std::unique_ptr read_buf_, - std::shared_ptr source_, - std::unique_ptr pipeline_, - std::unique_ptr reader_) - : relative_path_with_metadata(std::move(relative_path_with_metadata_)) - , read_buf(std::move(read_buf_)) - , source(std::move(source_)) - , pipeline(std::move(pipeline_)) - , reader(std::move(reader_)) - { - } - - ReaderHolder() = default; - ReaderHolder(const ReaderHolder & other) = delete; - ReaderHolder & operator=(const ReaderHolder & other) = delete; - - ReaderHolder(ReaderHolder && other) noexcept - { - *this = std::move(other); - } - - ReaderHolder & operator=(ReaderHolder && other) noexcept - { - /// The order of destruction is important. - /// reader uses pipeline, pipeline uses read_buf. - reader = std::move(other.reader); - pipeline = std::move(other.pipeline); - source = std::move(other.source); - read_buf = std::move(other.read_buf); - relative_path_with_metadata = std::move(other.relative_path_with_metadata); - return *this; - } - - explicit operator bool() const { return reader != nullptr; } - PullingPipelineExecutor * operator->() { return reader.get(); } - const PullingPipelineExecutor * operator->() const { return reader.get(); } - const String & getRelativePath() const { return relative_path_with_metadata.relative_path; } - const RelativePathWithMetadata & getRelativePathWithMetadata() const { return relative_path_with_metadata; } - const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } - - private: - RelativePathWithMetadata relative_path_with_metadata; - std::unique_ptr read_buf; - std::shared_ptr source; - std::unique_ptr pipeline; - std::unique_ptr reader; - }; - - ReaderHolder reader; - - LoggerPtr log = getLogger("StorageAzureBlobSource"); - - ThreadPool create_reader_pool; - ThreadPoolCallbackRunnerUnsafe create_reader_scheduler; - std::future reader_future; - - /// Recreate ReadBuffer and Pipeline for each file. - ReaderHolder createReader(); - std::future createReaderAsync(); - - std::unique_ptr createAzureReadBuffer(const String & key, size_t object_size); - std::unique_ptr createAsyncAzureReadBuffer( - const String & key, const ReadSettings & read_settings, size_t object_size); -}; - -} - -#endif diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp deleted file mode 100644 index a80d121567a..00000000000 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ /dev/null @@ -1,91 +0,0 @@ -#include "Storages/StorageAzureBlobCluster.h" - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -StorageAzureBlobCluster::StorageAzureBlobCluster( - const String & cluster_name_, - const StorageAzureBlob::Configuration & configuration_, - std::unique_ptr && object_storage_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const ContextPtr & context) - : IStorageCluster(cluster_name_, table_id_, getLogger("StorageAzureBlobCluster (" + table_id_.table_name + ")")) - , configuration{configuration_} - , object_storage(std::move(object_storage_)) -{ - context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.getConnectionURL()); - StorageInMemoryMetadata storage_metadata; - - if (columns_.empty()) - { - ColumnsDescription columns; - /// `format_settings` is set to std::nullopt, because StorageAzureBlobCluster is used only as table function - if (configuration.format == "auto") - std::tie(columns, configuration.format) = StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context); - else - columns = StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context); - storage_metadata.setColumns(columns); - } - else - { - if (configuration.format == "auto") - configuration.format = StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context).second; - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); -} - -void StorageAzureBlobCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) -{ - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query)); - - TableFunctionAzureBlobStorageCluster::updateStructureAndFormatArgumentsIfNeeded( - expression_list->children, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), configuration.format, context); -} - -RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const -{ - auto iterator = std::make_shared( - object_storage.get(), configuration.container, configuration.blob_path, - predicate, getVirtualsList(), context, nullptr); - - auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next().relative_path; }); - return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; -} - -} - -#endif diff --git a/src/Storages/StorageAzureBlobCluster.h b/src/Storages/StorageAzureBlobCluster.h deleted file mode 100644 index aca9630b8bf..00000000000 --- a/src/Storages/StorageAzureBlobCluster.h +++ /dev/null @@ -1,54 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include - -#include "Client/Connection.h" -#include -#include -#include - -namespace DB -{ - -class Context; - -class StorageAzureBlobCluster : public IStorageCluster -{ -public: - StorageAzureBlobCluster( - const String & cluster_name_, - const StorageAzureBlob::Configuration & configuration_, - std::unique_ptr && object_storage_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const ContextPtr & context); - - std::string getName() const override { return "AzureBlobStorageCluster"; } - - RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; - - bool supportsSubcolumns() const override { return true; } - - bool supportsDynamicSubcolumns() const override { return true; } - - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } - -private: - void updateBeforeRead(const ContextPtr & /*context*/) override {} - - void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override; - - StorageAzureBlob::Configuration configuration; - std::unique_ptr object_storage; -}; - - -} - -#endif diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index d9a0b2b4d59..a3f6b6afc5d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -302,6 +302,8 @@ void StorageBuffer::read( auto src_table_query_info = query_info; if (src_table_query_info.prewhere_info) { + src_table_query_info.prewhere_info = src_table_query_info.prewhere_info->clone(); + auto actions_dag = ActionsDAG::makeConvertingActions( header_after_adding_defaults.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp deleted file mode 100644 index 2ce188c203c..00000000000 --- a/src/Storages/StorageS3.cpp +++ /dev/null @@ -1,2311 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include "Common/logger_useful.h" -#include "IO/CompressionMethod.h" -#include "IO/ReadBuffer.h" -#include "Interpreters/Context_fwd.h" -#include "Storages/MergeTree/ReplicatedMergeTreePartHeader.h" - -#if USE_AWS_S3 - -#include - -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include - - -#include -#include -#include - -#include - -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -#include - -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" -#include -#pragma clang diagnostic pop - -namespace fs = std::filesystem; - - -namespace CurrentMetrics -{ - extern const Metric StorageS3Threads; - extern const Metric StorageS3ThreadsActive; - extern const Metric StorageS3ThreadsScheduled; -} - -namespace ProfileEvents -{ - extern const Event S3DeleteObjects; - extern const Event S3ListObjects; - extern const Event EngineFileLikeReadFiles; -} - -namespace DB -{ - -static const std::unordered_set required_configuration_keys = { - "url", -}; -static const std::unordered_set optional_configuration_keys = { - "format", - "compression", - "compression_method", - "structure", - "access_key_id", - "secret_access_key", - "session_token", - "filename", - "use_environment_credentials", - "max_single_read_retries", - "min_upload_part_size", - "upload_part_size_multiply_factor", - "upload_part_size_multiply_parts_count_threshold", - "max_single_part_upload_size", - "max_connections", - "expiration_window_seconds", - "no_sign_request" -}; - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_TEXT; - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int S3_ERROR; - extern const int UNEXPECTED_EXPRESSION; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int CANNOT_DETECT_FORMAT; - extern const int NOT_IMPLEMENTED; - extern const int CANNOT_COMPILE_REGEXP; - extern const int FILE_DOESNT_EXIST; - extern const int NO_ELEMENTS_IN_CONFIG; -} - - -class ReadFromStorageS3Step : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromStorageS3Step"; } - - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void applyFilters(ActionDAGNodes added_filter_nodes) override; - - ReadFromStorageS3Step( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - StorageS3 & storage_, - ReadFromFormatInfo read_from_format_info_, - bool need_only_count_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) - , column_names(column_names_) - , storage(storage_) - , read_from_format_info(std::move(read_from_format_info_)) - , need_only_count(need_only_count_) - , query_configuration(storage.getConfigurationCopy()) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - { - query_configuration.update(context); - virtual_columns = storage.getVirtualsList(); - } - -private: - Names column_names; - StorageS3 & storage; - ReadFromFormatInfo read_from_format_info; - bool need_only_count; - StorageS3::Configuration query_configuration; - NamesAndTypesList virtual_columns; - - size_t max_block_size; - size_t num_streams; - - std::shared_ptr iterator_wrapper; - - void createIterator(const ActionsDAG::Node * predicate); -}; - - -class IOutputFormat; -using OutputFormatPtr = std::shared_ptr; - -class StorageS3Source::DisclosedGlobIterator::Impl : WithContext -{ -public: - Impl( - const S3::Client & client_, - const S3::URI & globbed_uri_, - const ActionsDAG::Node * predicate_, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - KeysWithInfo * read_keys_, - const S3Settings::RequestSettings & request_settings_, - std::function file_progress_callback_) - : WithContext(context_) - , client(client_.clone()) - , globbed_uri(globbed_uri_) - , predicate(predicate_) - , virtual_columns(virtual_columns_) - , read_keys(read_keys_) - , request_settings(request_settings_) - , list_objects_pool( - CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, 1) - , list_objects_scheduler(threadPoolCallbackRunnerUnsafe(list_objects_pool, "ListObjects")) - , file_progress_callback(file_progress_callback_) - { - if (globbed_uri.bucket.find_first_of("*?{") != std::string::npos) - throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); - - expanded_keys = expandSelectionGlob(globbed_uri.key); - expanded_keys_iter = expanded_keys.begin(); - - fillBufferForKey(*expanded_keys_iter); - expanded_keys_iter++; - } - - KeyWithInfoPtr next(size_t) - { - std::lock_guard lock(mutex); - return nextAssumeLocked(); - } - - size_t objectsCount() - { - return buffer.size(); - } - - bool hasMore() - { - if (buffer.empty()) - return !(expanded_keys_iter == expanded_keys.end() && is_finished_for_key); - else - return true; - } - - ~Impl() - { - list_objects_pool.wait(); - } - -private: - using ListObjectsOutcome = Aws::S3::Model::ListObjectsV2Outcome; - - void fillBufferForKey(const std::string & uri_key) - { - is_finished_for_key = false; - const String key_prefix = uri_key.substr(0, uri_key.find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == uri_key.size()) - { - buffer.clear(); - buffer.emplace_back(std::make_shared(uri_key, std::nullopt)); - buffer_iter = buffer.begin(); - if (read_keys) - read_keys->insert(read_keys->end(), buffer.begin(), buffer.end()); - is_finished_for_key = true; - return; - } - - request = {}; - request.SetBucket(globbed_uri.bucket); - request.SetPrefix(key_prefix); - request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); - - outcome_future = listObjectsAsync(); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(uri_key)); - if (!matcher->ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", uri_key, matcher->error()); - - recursive = globbed_uri.key == "/**"; - - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - fillInternalBufferAssumeLocked(); - } - - KeyWithInfoPtr nextAssumeLocked() - { - do - { - if (buffer_iter != buffer.end()) - { - auto answer = *buffer_iter; - ++buffer_iter; - - /// If url doesn't contain globs, we didn't list s3 bucket and didn't get object info for the key. - /// So we get object info lazily here on 'next()' request. - if (!answer->info) - { - try - { - answer->info = S3::getObjectInfo(*client, globbed_uri.bucket, answer->key, globbed_uri.version_id, request_settings); - } - catch (...) - { - /// if no such file AND there was no `{}` glob -- this is an exception - /// otherwise ignore it, this is acceptable - if (expanded_keys.size() == 1) - throw; - continue; - } - if (file_progress_callback) - file_progress_callback(FileProgress(0, answer->info->size)); - } - - return answer; - } - - if (is_finished_for_key) - { - if (expanded_keys_iter != expanded_keys.end()) - { - fillBufferForKey(*expanded_keys_iter); - expanded_keys_iter++; - continue; - } - else - return {}; - } - - try - { - fillInternalBufferAssumeLocked(); - } - catch (...) - { - /// In case of exception thrown while listing new batch of files - /// iterator may be partially initialized and its further using may lead to UB. - /// Iterator is used by several processors from several threads and - /// it may take some time for threads to stop processors and they - /// may still use this iterator after exception is thrown. - /// To avoid this UB, reset the buffer and return defaults for further calls. - is_finished_for_key = true; - buffer.clear(); - buffer_iter = buffer.begin(); - throw; - } - } while (true); - } - - void fillInternalBufferAssumeLocked() - { - buffer.clear(); - assert(outcome_future.valid()); - auto outcome = outcome_future.get(); - - if (!outcome.IsSuccess()) - { - throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", - quoteString(request.GetBucket()), quoteString(request.GetPrefix()), - backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); - } - - const auto & result_batch = outcome.GetResult().GetContents(); - - /// It returns false when all objects were returned - is_finished_for_key = !outcome.GetResult().GetIsTruncated(); - - if (!is_finished_for_key) - { - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - list_objects_pool.wait(); - outcome_future = listObjectsAsync(); - } - - if (request_settings.throw_on_zero_files_match && result_batch.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files using prefix {}", request.GetPrefix()); - - KeysWithInfo temp_buffer; - temp_buffer.reserve(result_batch.size()); - - for (const auto & row : result_batch) - { - String key = row.GetKey(); - if (recursive || re2::RE2::FullMatch(key, *matcher)) - { - S3::ObjectInfo info = - { - .size = size_t(row.GetSize()), - .last_modification_time = row.GetLastModified().Millis() / 1000, - }; - - temp_buffer.emplace_back(std::make_shared(std::move(key), std::move(info))); - } - } - - if (temp_buffer.empty()) - { - buffer_iter = buffer.begin(); - return; - } - - if (filter_dag) - { - std::vector paths; - paths.reserve(temp_buffer.size()); - for (const auto & key_with_info : temp_buffer) - paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key); - - VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, filter_dag, virtual_columns, getContext()); - } - - buffer = std::move(temp_buffer); - - if (file_progress_callback) - { - for (const auto & key_with_info : buffer) - file_progress_callback(FileProgress(0, key_with_info->info->size)); - } - - /// Set iterator only after the whole batch is processed - buffer_iter = buffer.begin(); - - if (read_keys) - read_keys->insert(read_keys->end(), buffer.begin(), buffer.end()); - } - - std::future listObjectsAsync() - { - return list_objects_scheduler([this] - { - ProfileEvents::increment(ProfileEvents::S3ListObjects); - auto outcome = client->ListObjectsV2(request); - - /// Outcome failure will be handled on the caller side. - if (outcome.IsSuccess()) - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - - return outcome; - }, Priority{}); - } - - std::mutex mutex; - - KeysWithInfo buffer; - KeysWithInfo::iterator buffer_iter; - - std::vector expanded_keys; - std::vector::iterator expanded_keys_iter; - - std::unique_ptr client; - S3::URI globbed_uri; - const ActionsDAG::Node * predicate; - ASTPtr query; - NamesAndTypesList virtual_columns; - ActionsDAGPtr filter_dag; - std::unique_ptr matcher; - bool recursive{false}; - bool is_finished_for_key{false}; - KeysWithInfo * read_keys; - - S3::ListObjectsV2Request request; - S3Settings::RequestSettings request_settings; - - ThreadPool list_objects_pool; - ThreadPoolCallbackRunnerUnsafe list_objects_scheduler; - std::future outcome_future; - std::function file_progress_callback; -}; - -StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - const ContextPtr & context, - KeysWithInfo * read_keys_, - const S3Settings::RequestSettings & request_settings_, - std::function file_progress_callback_) - : pimpl(std::make_shared( - client_, globbed_uri_, predicate, virtual_columns_, context, read_keys_, request_settings_, file_progress_callback_)) -{ -} - -StorageS3Source::KeyWithInfoPtr StorageS3Source::DisclosedGlobIterator::next(size_t idx) /// NOLINT -{ - return pimpl->next(idx); -} - -size_t StorageS3Source::DisclosedGlobIterator::estimatedKeysCount() -{ - if (pimpl->hasMore()) - { - /// 1000 files were listed, and we cannot make any estimation of _how many more_ there are (because we list bucket lazily); - /// If there are more objects in the bucket, limiting the number of streams is the last thing we may want to do - /// as it would lead to serious slow down of the execution, since objects are going - /// to be fetched sequentially rather than in-parallel with up to times. - return std::numeric_limits::max(); - } - else - return pimpl->objectsCount(); -} - -class StorageS3Source::KeysIterator::Impl -{ -public: - explicit Impl( - const S3::Client & client_, - const std::string & version_id_, - const std::vector & keys_, - const String & bucket_, - const S3Settings::RequestSettings & request_settings_, - KeysWithInfo * read_keys_, - std::function file_progress_callback_) - : keys(keys_) - , client(client_.clone()) - , version_id(version_id_) - , bucket(bucket_) - , request_settings(request_settings_) - , file_progress_callback(file_progress_callback_) - { - if (read_keys_) - { - for (const auto & key : keys) - read_keys_->push_back(std::make_shared(key)); - } - } - - KeyWithInfoPtr next(size_t) - { - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= keys.size()) - return {}; - auto key = keys[current_index]; - std::optional info; - if (file_progress_callback) - { - info = S3::getObjectInfo(*client, bucket, key, version_id, request_settings); - file_progress_callback(FileProgress(0, info->size)); - } - - return std::make_shared(key, info); - } - - size_t objectsCount() - { - return keys.size(); - } - -private: - Strings keys; - std::atomic_size_t index = 0; - std::unique_ptr client; - String version_id; - String bucket; - S3Settings::RequestSettings request_settings; - std::function file_progress_callback; -}; - -StorageS3Source::KeysIterator::KeysIterator( - const S3::Client & client_, - const std::string & version_id_, - const std::vector & keys_, - const String & bucket_, - const S3Settings::RequestSettings & request_settings_, - KeysWithInfo * read_keys, - std::function file_progress_callback_) - : pimpl(std::make_shared( - client_, version_id_, keys_, bucket_, request_settings_, read_keys, file_progress_callback_)) -{ -} - -StorageS3Source::KeyWithInfoPtr StorageS3Source::KeysIterator::next(size_t idx) /// NOLINT -{ - return pimpl->next(idx); -} - -size_t StorageS3Source::KeysIterator::estimatedKeysCount() -{ - return pimpl->objectsCount(); -} - -StorageS3Source::ReadTaskIterator::ReadTaskIterator( - const DB::ReadTaskCallback & callback_, - size_t max_threads_count) - : callback(callback_) -{ - ThreadPool pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, max_threads_count); - auto pool_scheduler = threadPoolCallbackRunnerUnsafe(pool, "S3ReadTaskItr"); - - std::vector> keys; - keys.reserve(max_threads_count); - for (size_t i = 0; i < max_threads_count; ++i) - keys.push_back(pool_scheduler([this] { return callback(); }, Priority{})); - - pool.wait(); - buffer.reserve(max_threads_count); - for (auto & key_future : keys) - buffer.emplace_back(std::make_shared(key_future.get())); -} - -StorageS3Source::KeyWithInfoPtr StorageS3Source::ReadTaskIterator::next(size_t) /// NOLINT -{ - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= buffer.size()) - return std::make_shared(callback()); - - while (current_index < buffer.size()) - { - if (const auto & key_info = buffer[current_index]; key_info && !key_info->key.empty()) - return buffer[current_index]; - - current_index = index.fetch_add(1, std::memory_order_relaxed); - } - - return nullptr; -} - -size_t StorageS3Source::ReadTaskIterator::estimatedKeysCount() -{ - return buffer.size(); -} - - -StorageS3Source::ArchiveIterator::ArchiveIterator( - std::unique_ptr basic_iterator_, - const std::string & archive_pattern_, - std::shared_ptr client_, - const String & bucket_, - const String & version_id_, - const S3Settings::RequestSettings & request_settings_, - ContextPtr context_, - KeysWithInfo * read_keys_) - : WithContext(context_) - , basic_iterator(std::move(basic_iterator_)) - , basic_key_with_info_ptr(nullptr) - , client(client_) - , bucket(bucket_) - , version_id(version_id_) - , request_settings(request_settings_) - , read_keys(read_keys_) -{ - if (archive_pattern_.find_first_of("*?{") != std::string::npos) - { - auto matcher = std::make_shared(makeRegexpPatternFromGlobs(archive_pattern_)); - if (!matcher->ok()) - throw Exception( - ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", archive_pattern_, matcher->error()); - filter = IArchiveReader::NameFilter{[matcher](const std::string & p) mutable { return re2::RE2::FullMatch(p, *matcher); }}; - } - else - { - path_in_archive = archive_pattern_; - } -} - -StorageS3Source::KeyWithInfoPtr StorageS3Source::ArchiveIterator::next(size_t) -{ - if (!path_in_archive.empty()) - { - std::unique_lock lock{take_next_mutex}; - while (true) - { - basic_key_with_info_ptr = basic_iterator->next(); - if (!basic_key_with_info_ptr) - return {}; - refreshArchiveReader(); - bool file_exists = archive_reader->fileExists(path_in_archive); - if (file_exists) - { - KeyWithInfoPtr archive_key_with_info - = std::make_shared(basic_key_with_info_ptr->key, std::nullopt, path_in_archive, archive_reader); - if (read_keys != nullptr) - read_keys->push_back(archive_key_with_info); - return archive_key_with_info; - } - } - } - else - { - std::unique_lock lock{take_next_mutex}; - while (true) - { - if (!file_enumerator) - { - basic_key_with_info_ptr = basic_iterator->next(); - if (!basic_key_with_info_ptr) - return {}; - refreshArchiveReader(); - file_enumerator = archive_reader->firstFile(); - if (!file_enumerator) - { - file_enumerator.reset(); - continue; - } - } - else if (!file_enumerator->nextFile()) - { - file_enumerator.reset(); - continue; - } - - String current_filename = file_enumerator->getFileName(); - bool satisfies = filter(current_filename); - if (satisfies) - { - KeyWithInfoPtr archive_key_with_info - = std::make_shared(basic_key_with_info_ptr->key, std::nullopt, current_filename, archive_reader); - if (read_keys != nullptr) - read_keys->push_back(archive_key_with_info); - return archive_key_with_info; - } - } - } -} - -size_t StorageS3Source::ArchiveIterator::estimatedKeysCount() -{ - return basic_iterator->estimatedKeysCount(); -} - -void StorageS3Source::ArchiveIterator::refreshArchiveReader() -{ - if (basic_key_with_info_ptr) - { - if (!basic_key_with_info_ptr->info) - { - basic_key_with_info_ptr->info = S3::getObjectInfo(*client, bucket, basic_key_with_info_ptr->key, version_id, request_settings); - } - archive_reader = createArchiveReader( - basic_key_with_info_ptr->key, - [key = basic_key_with_info_ptr->key, archive_size = basic_key_with_info_ptr->info.value().size, context = getContext(), this]() - { return createS3ReadBuffer(key, archive_size, context, client, bucket, version_id, request_settings); }, - basic_key_with_info_ptr->info.value().size); - } - else - { - archive_reader = nullptr; - } -} - -StorageS3Source::StorageS3Source( - const ReadFromFormatInfo & info, - const String & format_, - String name_, - const ContextPtr & context_, - std::optional format_settings_, - UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, - String compression_hint_, - const std::shared_ptr & client_, - const String & bucket_, - const String & version_id_, - const String & url_host_and_port_, - std::shared_ptr file_iterator_, - const size_t max_parsing_threads_, - bool need_only_count_) - : SourceWithKeyCondition(info.source_header, false) - , WithContext(context_) - , name(std::move(name_)) - , bucket(bucket_) - , version_id(version_id_) - , url_host_and_port(url_host_and_port_) - , format(format_) - , columns_desc(info.columns_description) - , requested_columns(info.requested_columns) - , max_block_size(max_block_size_) - , request_settings(request_settings_) - , compression_hint(std::move(compression_hint_)) - , client(client_) - , sample_block(info.format_header) - , format_settings(format_settings_) - , requested_virtual_columns(info.requested_virtual_columns) - , file_iterator(file_iterator_) - , max_parsing_threads(max_parsing_threads_) - , need_only_count(need_only_count_) - , create_reader_pool( - CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, 1) - , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(create_reader_pool, "CreateS3Reader")) -{ -} - -void StorageS3Source::lazyInitialize(size_t idx) -{ - if (initialized) - return; - - reader = createReader(idx); - if (reader) - reader_future = createReaderAsync(idx); - initialized = true; -} - -StorageS3Source::ReaderHolder StorageS3Source::createReader(size_t idx) -{ - KeyWithInfoPtr key_with_info; - do - { - key_with_info = file_iterator->next(idx); - if (!key_with_info || key_with_info->key.empty()) - return {}; - - if (!key_with_info->info) - key_with_info->info = S3::getObjectInfo(*client, bucket, key_with_info->key, version_id, request_settings); - } - while (getContext()->getSettingsRef().s3_skip_empty_files && key_with_info->info->size == 0); - - QueryPipelineBuilder builder; - std::shared_ptr source; - std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(*key_with_info) : std::nullopt; - if (num_rows_from_cache) - { - /// We should not return single chunk with all number of rows, - /// because there is a chance that this chunk will be materialized later - /// (it can cause memory problems even with default values in columns or when virtual columns are requested). - /// Instead, we use special ConstChunkGenerator that will generate chunks - /// with max_block_size rows until total number of rows is reached. - source = std::make_shared(sample_block, *num_rows_from_cache, max_block_size); - builder.init(Pipe(source)); - } - else - { - auto compression_method = CompressionMethod::None; - if (!key_with_info->path_in_archive.has_value()) - { - compression_method = chooseCompressionMethod(key_with_info->key, compression_hint); - read_buf = createS3ReadBuffer( - key_with_info->key, key_with_info->info->size, getContext(), client, bucket, version_id, request_settings); - } - else - { - compression_method = chooseCompressionMethod(key_with_info->path_in_archive.value(), compression_hint); - read_buf = key_with_info->archive_reader->readFile(key_with_info->path_in_archive.value(), /*throw_on_not_found=*/true); - } - auto input_format = FormatFactory::instance().getInput( - format, - *read_buf, - sample_block, - getContext(), - max_block_size, - format_settings, - max_parsing_threads, - /* max_download_threads= */ std::nullopt, - /* is_remote_fs */ true, - compression_method, - need_only_count); - - if (key_condition) - input_format->setKeyCondition(key_condition); - - if (need_only_count) - input_format->needOnlyCount(); - - builder.init(Pipe(input_format)); - - if (columns_desc.hasDefaults()) - { - builder.addSimpleTransform( - [&](const Block & header) - { return std::make_shared(header, columns_desc, *input_format, getContext()); }); - } - - source = input_format; - } - - /// Add ExtractColumnsTransform to extract requested columns/subcolumns - /// from chunk read by IInputFormat. - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, requested_columns); - }); - - auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - auto current_reader = std::make_unique(*pipeline); - - ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - - return ReaderHolder{key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; -} - -std::future StorageS3Source::createReaderAsync(size_t idx) -{ - return create_reader_scheduler([=, this] { return createReader(idx); }, Priority{}); -} - -std::unique_ptr createS3ReadBuffer( - const String & key, - size_t object_size, - std::shared_ptr context, - std::shared_ptr client_ptr, - const String & bucket, - const String & version_id, - const S3Settings::RequestSettings & request_settings) -{ - auto read_settings = context->getReadSettings().adjustBufferSize(object_size); - read_settings.enable_filesystem_cache = false; - auto download_buffer_size = context->getSettings().max_download_buffer_size; - const bool object_too_small = object_size <= 2 * download_buffer_size; - static LoggerPtr log = getLogger("StorageS3Source"); - - // Create a read buffer that will prefetch the first ~1 MB of the file. - // When reading lots of tiny files, this prefetching almost doubles the throughput. - // For bigger files, parallel reading is more useful. - if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); - return createAsyncS3ReadBuffer(key, read_settings, object_size, context, client_ptr, bucket, version_id, request_settings); - } - - - return std::make_unique( - client_ptr, - bucket, - key, - version_id, - request_settings, - read_settings, - /*use_external_buffer*/ false, - /*offset_*/ 0, - /*read_until_position_*/ 0, - /*restricted_seek_*/ false, - object_size); -} - -std::unique_ptr createAsyncS3ReadBuffer( - const String & key, - const ReadSettings & read_settings, - size_t object_size, - std::shared_ptr context, - std::shared_ptr client_ptr, - const String & bucket, - const String & version_id, - const S3Settings::RequestSettings & request_settings) -{ - auto read_buffer_creator = [=](bool restricted_seek, const StoredObject & object) -> std::unique_ptr - { - return std::make_unique( - client_ptr, - bucket, - object.remote_path, - version_id, - request_settings, - read_settings, - /* use_external_buffer */ true, - /* offset */ 0, - /* read_until_position */ 0, - restricted_seek, - object_size); - }; - - auto modified_settings{read_settings}; - /// User's S3 object may change, don't cache it. - modified_settings.use_page_cache_for_disks_without_file_cache = false; - - /// FIXME: Changing this setting to default value breaks something around parquet reading - modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; - - auto s3_impl = std::make_unique( - std::move(read_buffer_creator), - StoredObjects{StoredObject{key, /* local_path */ "", object_size}}, - "", - read_settings, - /* cache_log */ nullptr, - /* use_external_buffer */ true); - - auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique( - std::move(s3_impl), pool_reader, modified_settings, context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog()); - - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; -} - -StorageS3Source::~StorageS3Source() -{ - create_reader_pool.wait(); -} - -String StorageS3Source::getName() const -{ - return name; -} - -Chunk StorageS3Source::generate() -{ - lazyInitialize(); - - while (true) - { - if (isCancelled() || !reader) - { - if (reader) - reader->cancel(); - break; - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - total_rows_in_file += num_rows; - size_t chunk_size = 0; - if (const auto * input_format = reader.getInputFormat()) - chunk_size = reader.getInputFormat()->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - String file_name = reader.getFile(); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, reader.getPath(), reader.getFileSize(), reader.isArchive() ? (&file_name) : nullptr); - return chunk; - } - - if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(reader.getPath(), total_rows_in_file); - - total_rows_in_file = 0; - - assert(reader_future.valid()); - reader = reader_future.get(); - - if (!reader) - break; - - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - create_reader_pool.wait(); - reader_future = createReaderAsync(); - } - - return {}; -} - -void StorageS3Source::addNumRowsToCache(const String & bucket_with_key, size_t num_rows) -{ - String source = fs::path(url_host_and_port) / bucket_with_key; - auto cache_key = getKeyForSchemaCache(source, format, format_settings, getContext()); - StorageS3::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); -} - -std::optional StorageS3Source::tryGetNumRowsFromCache(const KeyWithInfo & key_with_info) -{ - String source = fs::path(url_host_and_port) / bucket / key_with_info.key; - auto cache_key = getKeyForSchemaCache(source, format, format_settings, getContext()); - auto get_last_mod_time = [&]() -> std::optional { return key_with_info.info->last_modification_time; }; - - return StorageS3::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); -} - -class StorageS3Sink : public SinkToStorage -{ -public: - StorageS3Sink( - const String & format, - const Block & sample_block_, - const ContextPtr & context, - std::optional format_settings_, - const CompressionMethod compression_method, - const StorageS3::Configuration & configuration_, - const String & bucket, - const String & key) - : SinkToStorage(sample_block_), sample_block(sample_block_), format_settings(format_settings_) - { - BlobStorageLogWriterPtr blob_log = nullptr; - if (auto blob_storage_log = context->getBlobStorageLog()) - { - blob_log = std::make_shared(std::move(blob_storage_log)); - blob_log->query_id = context->getCurrentQueryId(); - } - - const auto & settings = context->getSettingsRef(); - write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique( - configuration_.client, - bucket, - key, - DBMS_DEFAULT_BUFFER_SIZE, - configuration_.request_settings, - std::move(blob_log), - std::nullopt, - threadPoolCallbackRunnerUnsafe(getIOThreadPool().get(), "S3ParallelWrite"), - context->getWriteSettings()), - compression_method, - static_cast(settings.output_format_compression_level), - static_cast(settings.output_format_compression_zstd_window_log)); - writer - = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); - } - - String getName() const override { return "StorageS3Sink"; } - - void consume(Chunk chunk) override - { - std::lock_guard lock(cancel_mutex); - if (cancelled) - return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); - } - - void onCancel() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - cancelled = true; - } - - void onException(std::exception_ptr exception) override - { - std::lock_guard lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization - release(); - } - } - - void onFinish() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - } - -private: - void finalize() - { - if (!writer) - return; - - try - { - writer->finalize(); - writer->flush(); - write_buf->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - release(); - throw; - } - } - - void release() - { - writer.reset(); - write_buf.reset(); - } - - Block sample_block; - std::optional format_settings; - std::unique_ptr write_buf; - OutputFormatPtr writer; - bool cancelled = false; - std::mutex cancel_mutex; -}; - -namespace -{ - -std::optional checkAndGetNewFileOnInsertIfNeeded( - const ContextPtr & context, const StorageS3::Configuration & configuration, const String & key, size_t sequence_number) -{ - if (context->getSettingsRef().s3_truncate_on_insert - || !S3::objectExists( - *configuration.client, configuration.url.bucket, key, configuration.url.version_id, configuration.request_settings)) - return std::nullopt; - - if (context->getSettingsRef().s3_create_new_file_on_insert) - { - auto pos = key.find_first_of('.'); - String new_key; - do - { - new_key = key.substr(0, pos) + "." + std::to_string(sequence_number) + (pos == std::string::npos ? "" : key.substr(pos)); - ++sequence_number; - } while (S3::objectExists( - *configuration.client, configuration.url.bucket, new_key, configuration.url.version_id, configuration.request_settings)); - - return new_key; - } - - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Object in bucket {} with key {} already exists. " - "If you want to overwrite it, enable setting s3_truncate_on_insert, if you " - "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", - configuration.url.bucket, key); -} -} - - -class PartitionedStorageS3Sink : public PartitionedSink, WithContext -{ -public: - PartitionedStorageS3Sink( - const ASTPtr & partition_by, - const String & format_, - const Block & sample_block_, - const ContextPtr & context_, - std::optional format_settings_, - const CompressionMethod compression_method_, - const StorageS3::Configuration & configuration_, - const String & bucket_, - const String & key_) - : PartitionedSink(partition_by, context_, sample_block_) - , WithContext(context_) - , format(format_) - , sample_block(sample_block_) - , compression_method(compression_method_) - , configuration(configuration_) - , bucket(bucket_) - , key(key_) - , format_settings(format_settings_) - { - } - - SinkPtr createSinkForPartition(const String & partition_id) override - { - auto partition_bucket = replaceWildcards(bucket, partition_id); - validateBucket(partition_bucket); - - auto partition_key = replaceWildcards(key, partition_id); - validateKey(partition_key); - - if (auto new_key = checkAndGetNewFileOnInsertIfNeeded(getContext(), configuration, partition_key, /* sequence_number */ 1)) - partition_key = *new_key; - - return std::make_shared( - format, sample_block, getContext(), format_settings, compression_method, configuration, partition_bucket, partition_key); - } - -private: - const String format; - const Block sample_block; - const CompressionMethod compression_method; - const StorageS3::Configuration configuration; - const String bucket; - const String key; - const std::optional format_settings; - - static void validateBucket(const String & str) - { - S3::URI::validateBucket(str, {}); - - if (!DB::UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in bucket name"); - - validatePartitionKey(str, false); - } - - static void validateKey(const String & str) - { - /// See: - /// - https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-keys.html - /// - https://cloud.ibm.com/apidocs/cos/cos-compatibility#putobject - - if (str.empty() || str.size() > 1024) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incorrect key length (not empty, max 1023 characters), got: {}", str.size()); - - if (!DB::UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in key"); - - validatePartitionKey(str, true); - } -}; - - -StorageS3::StorageS3( - const Configuration & configuration_, - const ContextPtr & context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_, - bool distributed_processing_, - ASTPtr partition_by_) - : IStorage(table_id_) - , configuration(configuration_) - , name(configuration.url.storage_name) - , distributed_processing(distributed_processing_) - , format_settings(format_settings_) - , partition_by(partition_by_) -{ - updateConfiguration(context_); // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall) - - if (configuration.format != "auto") - FormatFactory::instance().checkFormatName(configuration.format); - context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri); - context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration.headers_from_ast); - - StorageInMemoryMetadata storage_metadata; - if (columns_.empty()) - { - ColumnsDescription columns; - if (configuration.format == "auto") - std::tie(columns, configuration.format) = getTableStructureAndFormatFromData(configuration, format_settings, context_); - else - columns = getTableStructureFromData(configuration, format_settings, context_); - - storage_metadata.setColumns(columns); - } - else - { - if (configuration.format == "auto") - configuration.format = getTableStructureAndFormatFromData(configuration, format_settings, context_).second; - - /// We don't allow special columns in S3 storage. - if (!columns_.hasOnlyOrdinary()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Table engine S3 doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL"); - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); -} - -static std::shared_ptr createFileIterator( - StorageS3::Configuration configuration, - bool distributed_processing, - ContextPtr local_context, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns, - StorageS3Source::KeysWithInfo * read_keys = nullptr, - std::function file_progress_callback = {}) -{ - if (distributed_processing) - { - return std::make_shared( - local_context->getReadTaskCallback(), local_context->getSettingsRef().max_threads); - } - else - { - auto basic_iterator = [&]() -> std::unique_ptr - { - StorageS3Source::KeysWithInfo * local_read_keys = configuration.url.archive_pattern.has_value() ? nullptr : read_keys; - if (configuration.withGlobs()) - { - /// Iterate through disclosed globs and make a source for each file - return std::make_unique( - *configuration.client, - configuration.url, - predicate, - virtual_columns, - local_context, - local_read_keys, - configuration.request_settings, - file_progress_callback); - } - else - { - Strings keys = configuration.keys; - auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - if (filter_dag) - { - std::vector paths; - paths.reserve(keys.size()); - for (const auto & key : keys) - paths.push_back(fs::path(configuration.url.bucket) / key); - VirtualColumnUtils::filterByPathOrFile(keys, paths, filter_dag, virtual_columns, local_context); - } - return std::make_unique( - *configuration.client, - configuration.url.version_id, - keys, - configuration.url.bucket, - configuration.request_settings, - local_read_keys, - file_progress_callback); - } - }(); - if (configuration.url.archive_pattern.has_value()) - { - return std::make_shared( - std::move(basic_iterator), - configuration.url.archive_pattern.value(), - configuration.client, - configuration.url.bucket, - configuration.url.version_id, - configuration.request_settings, - local_context, - read_keys); - } - else - { - return basic_iterator; - } - } -} - -bool StorageS3::supportsSubsetOfColumns(const ContextPtr & context) const -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getFormatCopy(), context, format_settings); -} - -bool StorageS3::prefersLargeBlocks() const -{ - return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(getFormatCopy()); -} - -bool StorageS3::parallelizeOutputAfterReading(ContextPtr context) const -{ - return FormatFactory::instance().checkParallelizeOutputAfterReading(getFormatCopy(), context); -} - -void StorageS3::read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr local_context, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t num_streams) -{ - updateConfiguration(local_context); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); - - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && local_context->getSettingsRef().optimize_count_from_files; - - auto reading = std::make_unique( - column_names, - query_info, - storage_snapshot, - local_context, - read_from_format_info.source_header, - *this, - std::move(read_from_format_info), - need_only_count, - max_block_size, - num_streams); - - query_plan.addStep(std::move(reading)); -} - -void ReadFromStorageS3Step::applyFilters(ActionDAGNodes added_filter_nodes) -{ - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - createIterator(predicate); -} - -void ReadFromStorageS3Step::createIterator(const ActionsDAG::Node * predicate) -{ - if (iterator_wrapper) - return; - - iterator_wrapper = createFileIterator( - storage.getConfigurationCopy(), - storage.distributed_processing, - context, - predicate, - virtual_columns, - nullptr, - context->getFileProgressCallback()); -} - -void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - if (storage.partition_by && query_configuration.withPartitionWildcard()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); - - createIterator(nullptr); - size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); - if (estimated_keys_count > 1) - num_streams = std::min(num_streams, estimated_keys_count); - else - { - /// The amount of keys (zero) was probably underestimated. We will keep one stream for this particular case. - num_streams = 1; - } - - const size_t max_threads = context->getSettingsRef().max_threads; - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); - - Pipes pipes; - pipes.reserve(num_streams); - for (size_t i = 0; i < num_streams; ++i) - { - auto source = std::make_shared( - read_from_format_info, - query_configuration.format, - storage.getName(), - context, - storage.format_settings, - max_block_size, - query_configuration.request_settings, - query_configuration.compression_method, - query_configuration.client, - query_configuration.url.bucket, - query_configuration.url.version_id, - query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()), - iterator_wrapper, - max_parsing_threads, - need_only_count); - - source->setKeyCondition(filter_actions_dag, context); - pipes.emplace_back(std::move(source)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(read_from_format_info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); -} - -SinkToStoragePtr StorageS3::write( - const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) -{ - auto query_configuration = updateConfigurationAndGetCopy(local_context); - auto key = query_configuration.keys.front(); - - if (query_configuration.withGlobsIgnorePartitionWildcard()) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "S3 key '{}' contains globs, so the table is in readonly mode", query_configuration.url.key); - - auto sample_block = metadata_snapshot->getSampleBlock(); - auto chosen_compression_method = chooseCompressionMethod(query_configuration.keys.back(), query_configuration.compression_method); - auto insert_query = std::dynamic_pointer_cast(query); - - auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && query_configuration.withPartitionWildcard(); - - if (is_partitioned_implementation) - { - return std::make_shared( - partition_by_ast, - query_configuration.format, - sample_block, - local_context, - format_settings, - chosen_compression_method, - query_configuration, - query_configuration.url.bucket, - key); - } - else - { - if (auto new_key = checkAndGetNewFileOnInsertIfNeeded(local_context, query_configuration, query_configuration.keys.front(), query_configuration.keys.size())) - { - std::lock_guard lock{configuration_update_mutex}; - query_configuration.keys.push_back(*new_key); - configuration.keys.push_back(*new_key); - key = *new_key; - } - - return std::make_shared( - query_configuration.format, - sample_block, - local_context, - format_settings, - chosen_compression_method, - query_configuration, - query_configuration.url.bucket, - key); - } -} - -void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) -{ - auto query_configuration = updateConfigurationAndGetCopy(local_context); - - if (query_configuration.withGlobs()) - { - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "S3 key '{}' contains globs, so the table is in readonly mode", - query_configuration.url.key); - } - - Aws::S3::Model::Delete delkeys; - - for (const auto & key : query_configuration.keys) - { - Aws::S3::Model::ObjectIdentifier obj; - obj.SetKey(key); - delkeys.AddObjects(std::move(obj)); - } - - ProfileEvents::increment(ProfileEvents::S3DeleteObjects); - S3::DeleteObjectsRequest request; - request.SetBucket(query_configuration.url.bucket); - request.SetDelete(delkeys); - - auto response = query_configuration.client->DeleteObjects(request); - - const auto * response_error = response.IsSuccess() ? nullptr : &response.GetError(); - auto time_now = std::chrono::system_clock::now(); - if (auto blob_storage_log = BlobStorageLogWriter::create()) - for (const auto & key : query_configuration.keys) - blob_storage_log->addEvent( - BlobStorageLogElement::EventType::Delete, query_configuration.url.bucket, key, {}, 0, response_error, time_now); - - if (!response.IsSuccess()) - { - const auto & err = response.GetError(); - throw S3Exception(err.GetMessage(), err.GetErrorType()); - } - - for (const auto & error : response.GetResult().GetErrors()) - LOG_WARNING(getLogger("StorageS3"), "Failed to delete {}, error: {}", error.GetKey(), error.GetMessage()); -} - -StorageS3::Configuration StorageS3::updateConfigurationAndGetCopy(const ContextPtr & local_context) -{ - std::lock_guard lock(configuration_update_mutex); - configuration.update(local_context); - return configuration; -} - -void StorageS3::updateConfiguration(const ContextPtr & local_context) -{ - std::lock_guard lock(configuration_update_mutex); - configuration.update(local_context); -} - -void StorageS3::useConfiguration(const StorageS3::Configuration & new_configuration) -{ - std::lock_guard lock(configuration_update_mutex); - configuration = new_configuration; -} - -StorageS3::Configuration StorageS3::getConfigurationCopy() const -{ - std::lock_guard lock(configuration_update_mutex); - return configuration; -} - -String StorageS3::getFormatCopy() const -{ - std::lock_guard lock(configuration_update_mutex); - return configuration.format; -} - -bool StorageS3::Configuration::update(const ContextPtr & context) -{ - auto s3_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName()); - request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context->getSettings()); - - if (client && (static_configuration || !auth_settings.hasUpdates(s3_settings.auth_settings))) - return false; - - auth_settings.updateFrom(s3_settings.auth_settings); - keys[0] = url.key; - connect(context); - return true; -} - -void StorageS3::Configuration::connect(const ContextPtr & context) -{ - const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); - const Settings & local_settings = context->getSettingsRef(); - - if (S3::isS3ExpressEndpoint(url.endpoint) && auth_settings.region.empty()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Region should be explicitly specified for directory buckets"); - - S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - auth_settings.region, - context->getRemoteHostFilter(), - static_cast(global_settings.s3_max_redirects), - static_cast(global_settings.s3_retry_attempts), - global_settings.enable_s3_requests_logging, - /* for_disk_s3 = */ false, - request_settings.get_request_throttler, - request_settings.put_request_throttler, - url.uri.getScheme()); - - client_configuration.endpointOverride = url.endpoint; - /// seems as we don't use it - client_configuration.maxConnections = static_cast(request_settings.max_connections); - client_configuration.connectTimeoutMs = local_settings.s3_connect_timeout_ms; - client_configuration.http_keep_alive_timeout = S3::DEFAULT_KEEP_ALIVE_TIMEOUT; - client_configuration.http_keep_alive_max_requests = S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS; - - auto headers = auth_settings.headers; - if (!headers_from_ast.empty()) - headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); - - client_configuration.requestTimeoutMs = request_settings.request_timeout_ms; - - S3::ClientSettings client_settings{ - .use_virtual_addressing = url.is_virtual_hosted_style, - .disable_checksum = local_settings.s3_disable_checksum, - .gcs_issue_compose_request = context->getConfigRef().getBool("s3.gcs_issue_compose_request", false), - .is_s3express_bucket = S3::isS3ExpressEndpoint(url.endpoint), - }; - - auto credentials - = Aws::Auth::AWSCredentials(auth_settings.access_key_id, auth_settings.secret_access_key, auth_settings.session_token); - client = S3::ClientFactory::instance().create( - client_configuration, - client_settings, - credentials.GetAWSAccessKeyId(), - credentials.GetAWSSecretKey(), - auth_settings.server_side_encryption_customer_key_base64, - auth_settings.server_side_encryption_kms_config, - std::move(headers), - S3::CredentialsConfiguration{ - auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), - auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - auth_settings.expiration_window_seconds.value_or( - context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), - auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), - }, - credentials.GetSessionToken()); -} - -bool StorageS3::Configuration::withGlobsIgnorePartitionWildcard() const -{ - if (!withPartitionWildcard()) - return withGlobs(); - - return PartitionedSink::replaceWildcards(getPath(), "").find_first_of("*?{") != std::string::npos; -} - -void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) -{ - validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); - - auto filename = collection.getOrDefault("filename", ""); - if (!filename.empty()) - configuration.url = S3::URI(std::filesystem::path(collection.get("url")) / filename); - else - configuration.url = S3::URI(collection.get("url")); - - configuration.auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); - configuration.auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); - configuration.auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 1); - configuration.auth_settings.no_sign_request = collection.getOrDefault("no_sign_request", false); - configuration.auth_settings.expiration_window_seconds - = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); - - configuration.format = collection.getOrDefault("format", configuration.format); - configuration.compression_method - = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); - configuration.structure = collection.getOrDefault("structure", "auto"); - - configuration.request_settings = S3Settings::RequestSettings(collection); -} - -StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file) -{ - StorageS3::Configuration configuration; - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - { - processNamedCollectionResult(configuration, *named_collection); - } - else - { - /// Supported signatures: - /// - /// S3('url') - /// S3('url', 'format') - /// S3('url', 'format', 'compression') - /// S3('url', NOSIGN) - /// S3('url', NOSIGN, 'format') - /// S3('url', NOSIGN, 'format', 'compression') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format', 'compression') - /// with optional headers() function - - size_t count = StorageURL::evalArgsAndCollectHeaders(engine_args, configuration.headers_from_ast, local_context); - - if (count == 0 || count > 6) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage S3 requires 1 to 6 positional arguments: " - "url, [NOSIGN | access_key_id, secret_access_key], [session_token], [name of used format], [compression_method], [headers], [extra_credentials]"); - - std::unordered_map engine_args_to_idx; - bool no_sign_request = false; - - /// For 2 arguments we support 2 possible variants: - /// - s3(source, format) - /// - s3(source, NOSIGN) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. - if (count == 2) - { - auto second_arg = checkAndGetLiteralArgument(engine_args[1], "format/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - no_sign_request = true; - else - engine_args_to_idx = {{"format", 1}}; - } - /// For 3 arguments we support 2 possible variants: - /// - s3(source, format, compression_method) - /// - s3(source, access_key_id, secret_access_key) - /// - s3(source, NOSIGN, format) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name. - else if (count == 3) - { - auto second_arg = checkAndGetLiteralArgument(engine_args[1], "format/access_key_id/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - engine_args_to_idx = {{"format", 2}}; - } - else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) - engine_args_to_idx = {{"format", 1}, {"compression_method", 2}}; - else - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; - } - /// For 4 arguments we support 3 possible variants: - /// - s3(source, access_key_id, secret_access_key, session_token) - /// - s3(source, access_key_id, secret_access_key, format) - /// - s3(source, NOSIGN, format, compression_method) - /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not. - else if (count == 4) - { - auto second_arg = checkAndGetLiteralArgument(engine_args[1], "access_key_id/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - engine_args_to_idx = {{"format", 2}, {"compression_method", 3}}; - } - else - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "session_token/format"); - if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; - else - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}}; - } - } - /// For 5 arguments we support 2 possible variants: - /// - s3(source, access_key_id, secret_access_key, session_token, format) - /// - s3(source, access_key_id, secret_access_key, format, compression) - else if (count == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "session_token/format"); - if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"compression", 4}}; - else - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}}; - } - else if (count == 6) - { - engine_args_to_idx - = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"compression_method", 5}}; - } - - /// This argument is always the first - configuration.url = S3::URI(checkAndGetLiteralArgument(engine_args[0], "url")); - - if (engine_args_to_idx.contains("format")) - configuration.format = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["format"]], "format"); - - if (engine_args_to_idx.contains("compression_method")) - configuration.compression_method - = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["compression_method"]], "compression_method"); - - if (engine_args_to_idx.contains("access_key_id")) - configuration.auth_settings.access_key_id - = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["access_key_id"]], "access_key_id"); - - if (engine_args_to_idx.contains("secret_access_key")) - configuration.auth_settings.secret_access_key - = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["secret_access_key"]], "secret_access_key"); - - if (engine_args_to_idx.contains("session_token")) - configuration.auth_settings.session_token - = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["session_token"]], "session_token"); - - if (no_sign_request) - configuration.auth_settings.no_sign_request = no_sign_request; - } - - configuration.static_configuration - = !configuration.auth_settings.access_key_id.empty() || configuration.auth_settings.no_sign_request.has_value(); - - configuration.keys = {configuration.url.key}; - - if (configuration.format == "auto" && get_format_from_file) - { - if (configuration.url.archive_pattern.has_value()) - { - configuration.format = FormatFactory::instance() - .tryGetFormatFromFileName(Poco::URI(configuration.url.archive_pattern.value()).getPath()) - .value_or("auto"); - } - else - { - configuration.format - = FormatFactory::instance().tryGetFormatFromFileName(Poco::URI(configuration.url.uri_str).getPath()).value_or("auto"); - } - } - - return configuration; -} - -ColumnsDescription StorageS3::getTableStructureFromData( - const StorageS3::Configuration & configuration_, const std::optional & format_settings_, const ContextPtr & ctx) -{ - return getTableStructureAndFormatFromDataImpl(configuration_.format, configuration_, format_settings_, ctx).first; -} - -std::pair StorageS3::getTableStructureAndFormatFromData( - const StorageS3::Configuration & configuration, const std::optional & format_settings, const ContextPtr & ctx) -{ - return getTableStructureAndFormatFromDataImpl(std::nullopt, configuration, format_settings, ctx); -} - -class ReadBufferIterator : public IReadBufferIterator, WithContext -{ -public: - ReadBufferIterator( - std::shared_ptr file_iterator_, - const StorageS3Source::KeysWithInfo & read_keys_, - const StorageS3::Configuration & configuration_, - std::optional format_, - const std::optional & format_settings_, - ContextPtr context_) - : WithContext(context_) - , file_iterator(file_iterator_) - , read_keys(read_keys_) - , configuration(configuration_) - , format(std::move(format_)) - , format_settings(format_settings_) - , prev_read_keys_size(read_keys_.size()) - { - } - - Data next() override - { - if (first) - { - /// If format is unknown we iterate through all currently read keys on first iteration and - /// try to determine format by file name. - if (!format) - { - for (const auto & key_with_info : read_keys) - { - if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(key_with_info->getFileName())) - { - format = format_from_file_name; - break; - } - } - } - - /// For default mode check cached columns for currently read keys on first iteration. - if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) - return {nullptr, cached_columns, format}; - } - } - - while (true) - { - current_key_with_info = (*file_iterator)(); - - if (!current_key_with_info || current_key_with_info->key.empty()) - { - if (first) - { - if (format) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The table structure cannot be extracted from a {} format file, because there are no files with provided path " - "in S3 or all files are empty. You can specify table structure manually", - *format); - - throw Exception( - ErrorCodes::CANNOT_DETECT_FORMAT, - "The data format cannot be detected by the contents of the files, because there are no files with provided path " - "in S3 or all files are empty. You can specify the format manually"); - } - - return {nullptr, std::nullopt, format}; - } - - if (read_keys.size() > prev_read_keys_size) - { - /// If format is unknown we can try to determine it by new file names. - if (!format) - { - for (auto it = read_keys.begin() + prev_read_keys_size; it != read_keys.end(); ++it) - { - if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->getFileName())) - { - format = format_from_file_name; - break; - } - } - } - - /// Check new files in schema cache if schema inference mode is default. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); - if (columns_from_cache) - return {nullptr, columns_from_cache, format}; - } - - prev_read_keys_size = read_keys.size(); - } - - if (getContext()->getSettingsRef().s3_skip_empty_files && current_key_with_info->info && current_key_with_info->info->size == 0) - continue; - - /// In union mode, check cached columns only for current key. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - StorageS3Source::KeysWithInfo keys = {current_key_with_info}; - if (auto columns_from_cache = tryGetColumnsFromCache(keys.begin(), keys.end())) - { - first = false; - return {nullptr, columns_from_cache, format}; - } - } - - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - std::unique_ptr impl; - - if (!current_key_with_info->path_in_archive.has_value()) - { - impl = std::make_unique( - configuration.client, - configuration.url.bucket, - current_key_with_info->key, - configuration.url.version_id, - configuration.request_settings, - getContext()->getReadSettings()); - } - else - { - assert(current_key_with_info->archive_reader); - impl = current_key_with_info->archive_reader->readFile( - current_key_with_info->path_in_archive.value(), /*throw_on_not_found=*/true); - } - if (!getContext()->getSettingsRef().s3_skip_empty_files || !impl->eof()) - { - first = false; - return { - wrapReadBufferWithCompressionMethod( - std::move(impl), - current_key_with_info->path_in_archive.has_value() - ? chooseCompressionMethod(current_key_with_info->path_in_archive.value(), configuration.compression_method) - : chooseCompressionMethod(current_key_with_info->key, configuration.compression_method), - zstd_window_log_max), - std::nullopt, - format}; - } - } - } - - void setNumRowsToLastFile(size_t num_rows) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) - return; - - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) - / configuration.url.bucket / current_key_with_info->getPath(); - auto key = getKeyForSchemaCache(source, *format, format_settings, getContext()); - StorageS3::getSchemaCache(getContext()).addNumRows(key, num_rows); - } - - void setSchemaToLastFile(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3 - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) - return; - - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) - / configuration.url.bucket / current_key_with_info->getPath(); - auto cache_key = getKeyForSchemaCache(source, *format, format_settings, getContext()); - StorageS3::getSchemaCache(getContext()).addColumns(cache_key, columns); - } - - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3 - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) - return; - - auto host_and_bucket = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket; - Strings sources; - sources.reserve(read_keys.size()); - std::transform( - read_keys.begin(), - read_keys.end(), - std::back_inserter(sources), - [&](const auto & elem) { return host_and_bucket / elem->getPath(); }); - auto cache_keys = getKeysForSchemaCache(sources, *format, format_settings, getContext()); - StorageS3::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); - } - - void setFormatName(const String & format_name) override - { - format = format_name; - } - - String getLastFileName() const override - { - if (current_key_with_info) - return current_key_with_info->getPath(); - return ""; - } - - bool supportsLastReadBufferRecreation() const override { return true; } - - std::unique_ptr recreateLastReadBuffer() override - { - chassert(current_key_with_info); - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - auto impl = std::make_unique(configuration.client, configuration.url.bucket, current_key_with_info->key, configuration.url.version_id, configuration.request_settings, getContext()->getReadSettings()); - return wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info->key, configuration.compression_method), zstd_window_log_max); - } - -private: - std::optional tryGetColumnsFromCache( - const StorageS3Source::KeysWithInfo::const_iterator & begin, const StorageS3Source::KeysWithInfo::const_iterator & end) - { - auto context = getContext(); - if (!context->getSettingsRef().schema_inference_use_cache_for_s3) - return std::nullopt; - - auto & schema_cache = StorageS3::getSchemaCache(context); - for (auto it = begin; it < end; ++it) - { - auto get_last_mod_time = [&] - { - time_t last_modification_time = 0; - if ((*it)->info) - { - last_modification_time = (*it)->info->last_modification_time; - } - else - { - /// Note that in case of exception in getObjectInfo returned info will be empty, - /// but schema cache will handle this case and won't return columns from cache - /// because we can't say that it's valid without last modification time. - last_modification_time = S3::getObjectInfo( - *configuration.client, - configuration.url.bucket, - (*it)->key, - configuration.url.version_id, - configuration.request_settings, - /*with_metadata=*/ false, - /*throw_on_error= */ false).last_modification_time; - } - - return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt; - }; - String path = fs::path(configuration.url.bucket) / (*it)->getPath(); - - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path; - - if (format) - { - auto cache_key = getKeyForSchemaCache(source, *format, format_settings, context); - if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) - return columns; - } - else - { - /// If format is unknown, we can iterate through all possible input formats - /// and check if we have an entry with this format and this file in schema cache. - /// If we have such entry fcreateor some format, we can use this format to read the file. - for (const auto & format_name : FormatFactory::instance().getAllInputFormats()) - { - auto cache_key = getKeyForSchemaCache(source, format_name, format_settings, context); - if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) - { - /// Now format is known. It should be the same for all files. - format = format_name; - return columns; - } - } - } - } - - return std::nullopt; - } - - std::shared_ptr file_iterator; - const StorageS3Source::KeysWithInfo & read_keys; - const StorageS3::Configuration & configuration; - std::optional format; - const std::optional & format_settings; - StorageS3Source::KeyWithInfoPtr current_key_with_info; - size_t prev_read_keys_size; - bool first = true; -}; - -std::pair StorageS3::getTableStructureAndFormatFromDataImpl( - std::optional format, - const StorageS3::Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx) -{ - KeysWithInfo read_keys; - - auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, &read_keys); - - ReadBufferIterator read_buffer_iterator(file_iterator, read_keys, configuration, format, format_settings, ctx); - if (format) - return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, ctx), *format}; - return detectFormatAndReadSchema(format_settings, read_buffer_iterator, ctx); -} - -void registerStorageS3Impl(const String & name, StorageFactory & factory) -{ - factory.registerStorage(name, [](const StorageFactory::Arguments & args) - { - auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); - // Use format settings from global server context + settings from - // the SETTINGS clause of the create query. Settings from current - // session and user are ignored. - std::optional format_settings; - if (args.storage_def->settings) - { - FormatFactorySettings user_format_settings; - - // Apply changed settings from global context, but ignore the - // unknown ones, because we only have the format settings here. - const auto & changes = args.getContext()->getSettingsRef().changes(); - for (const auto & change : changes) - { - if (user_format_settings.has(change.name)) - user_format_settings.set(change.name, change.value); - } - - // Apply changes from SETTINGS clause, with validation. - user_format_settings.applyChanges(args.storage_def->settings->changes); - format_settings = getFormatSettings(args.getContext(), user_format_settings); - } - else - { - format_settings = getFormatSettings(args.getContext()); - } - - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - - return std::make_shared( - std::move(configuration), - args.getContext(), - args.table_id, - args.columns, - args.constraints, - args.comment, - format_settings, - /* distributed_processing_ */false, - partition_by); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); -} - -void registerStorageS3(StorageFactory & factory) -{ - registerStorageS3Impl("S3", factory); - registerStorageS3Impl("COSN", factory); - registerStorageS3Impl("OSS", factory); -} - -bool StorageS3::supportsPartitionBy() const -{ - return true; -} - -SchemaCache & StorageS3::getSchemaCache(const ContextPtr & ctx) -{ - static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_s3", DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; -} -} - -#endif diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h deleted file mode 100644 index dd6d13b99cb..00000000000 --- a/src/Storages/StorageS3.h +++ /dev/null @@ -1,464 +0,0 @@ -#pragma once - -#include -#include -#include "IO/Archives/IArchiveReader.h" -#include "IO/Archives/createArchiveReader.h" -#include "IO/ReadBuffer.h" -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int LOGICAL_ERROR; -} - -class PullingPipelineExecutor; -class NamedCollection; - -class StorageS3Source : public SourceWithKeyCondition, WithContext -{ -public: - struct KeyWithInfo - { - KeyWithInfo() = default; - - explicit KeyWithInfo( - String key_, - std::optional info_ = std::nullopt, - std::optional path_in_archive_ = std::nullopt, - std::shared_ptr archive_reader_ = nullptr) - : key(std::move(key_)) - , info(std::move(info_)) - , path_in_archive(std::move(path_in_archive_)) - , archive_reader(std::move(archive_reader_)) - { - if (path_in_archive.has_value() != (archive_reader != nullptr)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Archive reader and path in archive must exist simultaneously"); - } - - virtual ~KeyWithInfo() = default; - - String key; - std::optional info; - std::optional path_in_archive; - std::shared_ptr archive_reader; - - String getPath() const { return path_in_archive.has_value() ? (key + "::" + path_in_archive.value()) : key; } - String getFileName() const { return path_in_archive.has_value() ? path_in_archive.value() : key; } - }; - - using KeyWithInfoPtr = std::shared_ptr; - - using KeysWithInfo = std::vector; - class IIterator - { - public: - virtual ~IIterator() = default; - virtual KeyWithInfoPtr next(size_t idx = 0) = 0; /// NOLINT - - /// Estimates how many streams we need to process all files. - /// If keys count >= max_threads_count, the returned number may not represent the actual number of the keys. - /// Intended to be called before any next() calls, may underestimate otherwise - /// fixme: May underestimate if the glob has a strong filter, so there are few matches among the first 1000 ListObjects results. - virtual size_t estimatedKeysCount() = 0; - - KeyWithInfoPtr operator()() { return next(); } - }; - - class DisclosedGlobIterator : public IIterator - { - public: - DisclosedGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns, - const ContextPtr & context, - KeysWithInfo * read_keys_ = nullptr, - const S3Settings::RequestSettings & request_settings_ = {}, - std::function progress_callback_ = {}); - - KeyWithInfoPtr next(size_t idx = 0) override; /// NOLINT - size_t estimatedKeysCount() override; - - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - class KeysIterator : public IIterator - { - public: - explicit KeysIterator( - const S3::Client & client_, - const std::string & version_id_, - const std::vector & keys_, - const String & bucket_, - const S3Settings::RequestSettings & request_settings_, - KeysWithInfo * read_keys = nullptr, - std::function progress_callback_ = {}); - - KeyWithInfoPtr next(size_t idx = 0) override; /// NOLINT - size_t estimatedKeysCount() override; - - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - class ReadTaskIterator : public IIterator - { - public: - explicit ReadTaskIterator(const ReadTaskCallback & callback_, size_t max_threads_count); - - KeyWithInfoPtr next(size_t idx = 0) override; /// NOLINT - size_t estimatedKeysCount() override; - - private: - KeysWithInfo buffer; - std::atomic_size_t index = 0; - - ReadTaskCallback callback; - }; - - class ArchiveIterator : public IIterator, public WithContext - { - public: - explicit ArchiveIterator( - std::unique_ptr basic_iterator_, - const std::string & archive_pattern_, - std::shared_ptr client_, - const String & bucket_, - const String & version_id_, - const S3Settings::RequestSettings & request_settings, - ContextPtr context_, - KeysWithInfo * read_keys_); - - KeyWithInfoPtr next(size_t) override; /// NOLINT - size_t estimatedKeysCount() override; - void refreshArchiveReader(); - - private: - std::unique_ptr basic_iterator; - KeyWithInfoPtr basic_key_with_info_ptr; - std::unique_ptr basic_read_buffer; - std::shared_ptr archive_reader{nullptr}; - std::unique_ptr file_enumerator = nullptr; - std::string path_in_archive = {}; // used when reading a single file from archive - IArchiveReader::NameFilter filter = {}; // used when files inside archive are defined with a glob - std::shared_ptr client; - const String bucket; - const String version_id; - S3Settings::RequestSettings request_settings; - std::mutex take_next_mutex; - KeysWithInfo * read_keys; - }; - - friend StorageS3Source::ArchiveIterator; - - StorageS3Source( - const ReadFromFormatInfo & info, - const String & format, - String name_, - const ContextPtr & context_, - std::optional format_settings_, - UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, - String compression_hint_, - const std::shared_ptr & client_, - const String & bucket, - const String & version_id, - const String & url_host_and_port, - std::shared_ptr file_iterator_, - size_t max_parsing_threads, - bool need_only_count_); - - ~StorageS3Source() override; - - String getName() const override; - - void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override - { - setKeyConditionImpl(filter_actions_dag, context_, sample_block); - } - - Chunk generate() override; - -private: - friend class StorageS3QueueSource; - - String name; - String bucket; - String version_id; - String url_host_and_port; - String format; - ColumnsDescription columns_desc; - NamesAndTypesList requested_columns; - UInt64 max_block_size; - S3Settings::RequestSettings request_settings; - String compression_hint; - std::shared_ptr client; - Block sample_block; - std::optional format_settings; - - struct ReaderHolder - { - public: - ReaderHolder( - KeyWithInfoPtr key_with_info_, - String bucket_, - std::unique_ptr read_buf_, - std::shared_ptr source_, - std::unique_ptr pipeline_, - std::unique_ptr reader_) - : key_with_info(key_with_info_) - , bucket(std::move(bucket_)) - , read_buf(std::move(read_buf_)) - , source(std::move(source_)) - , pipeline(std::move(pipeline_)) - , reader(std::move(reader_)) - { - } - - ReaderHolder() = default; - ReaderHolder(const ReaderHolder & other) = delete; - ReaderHolder & operator=(const ReaderHolder & other) = delete; - - ReaderHolder(ReaderHolder && other) noexcept { *this = std::move(other); } - - ReaderHolder & operator=(ReaderHolder && other) noexcept - { - /// The order of destruction is important. - /// reader uses pipeline, pipeline uses read_buf. - reader = std::move(other.reader); - pipeline = std::move(other.pipeline); - source = std::move(other.source); - read_buf = std::move(other.read_buf); - key_with_info = std::move(other.key_with_info); - bucket = std::move(other.bucket); - return *this; - } - - explicit operator bool() const { return reader != nullptr; } - PullingPipelineExecutor * operator->() { return reader.get(); } - const PullingPipelineExecutor * operator->() const { return reader.get(); } - String getPath() const { return bucket + "/" + key_with_info->getPath(); } - String getFile() const { return key_with_info->getFileName(); } - bool isArchive() { return key_with_info->path_in_archive.has_value(); } - const KeyWithInfo & getKeyWithInfo() const { return *key_with_info; } - std::optional getFileSize() const { return key_with_info->info ? std::optional(key_with_info->info->size) : std::nullopt; } - - const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } - - private: - KeyWithInfoPtr key_with_info; - String bucket; - std::unique_ptr read_buf; - std::shared_ptr source; - std::unique_ptr pipeline; - std::unique_ptr reader; - }; - - ReaderHolder reader; - - NamesAndTypesList requested_virtual_columns; - std::shared_ptr file_iterator; - size_t max_parsing_threads = 1; - bool need_only_count; - - LoggerPtr log = getLogger("StorageS3Source"); - - ThreadPool create_reader_pool; - ThreadPoolCallbackRunnerUnsafe create_reader_scheduler; - std::future reader_future; - std::atomic initialized{false}; - - size_t total_rows_in_file = 0; - - /// Notice: we should initialize reader and future_reader lazily in generate to make sure key_condition - /// is set before createReader is invoked for key_condition is read in createReader. - void lazyInitialize(size_t idx = 0); - - /// Recreate ReadBuffer and Pipeline for each file. - ReaderHolder createReader(size_t idx = 0); - std::future createReaderAsync(size_t idx = 0); - - void addNumRowsToCache(const String & bucket_with_key, size_t num_rows); - std::optional tryGetNumRowsFromCache(const KeyWithInfo & key_with_info); -}; - -/** - * This class represents table engine for external S3 urls. - * It sends HTTP GET to server when select is called and - * HTTP PUT when insert is called. - */ -class StorageS3 : public IStorage -{ -public: - struct Configuration : public StatelessTableEngineConfiguration - { - Configuration() = default; - - const String & getPath() const { return url.key; } - - bool update(const ContextPtr & context); - - void connect(const ContextPtr & context); - - bool withGlobs() const { return url.key.find_first_of("*?{") != std::string::npos; } - - bool withPartitionWildcard() const - { - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - return url.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; - } - - bool withGlobsIgnorePartitionWildcard() const; - - S3::URI url; - S3::AuthSettings auth_settings; - S3Settings::RequestSettings request_settings; - /// If s3 configuration was passed from ast, then it is static. - /// If from config - it can be changed with config reload. - bool static_configuration = true; - /// Headers from ast is a part of static configuration. - HTTPHeaderEntries headers_from_ast; - - std::shared_ptr client; - std::vector keys; - }; - - StorageS3( - const Configuration & configuration_, - const ContextPtr & context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_, - bool distributed_processing_ = false, - ASTPtr partition_by_ = nullptr); - - String getName() const override { return name; } - - void read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams) override; - - SinkToStoragePtr - write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; - - void truncate( - const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; - - bool supportsPartitionBy() const override; - - static void processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection); - - static SchemaCache & getSchemaCache(const ContextPtr & ctx); - - static Configuration getConfiguration(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file = true); - - static ColumnsDescription getTableStructureFromData( - const Configuration & configuration_, const std::optional & format_settings_, const ContextPtr & ctx); - - static std::pair getTableStructureAndFormatFromData( - const Configuration & configuration, const std::optional & format_settings, const ContextPtr & ctx); - - using KeysWithInfo = StorageS3Source::KeysWithInfo; - - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } - -protected: - virtual Configuration updateConfigurationAndGetCopy(const ContextPtr & local_context); - - virtual void updateConfiguration(const ContextPtr & local_context); - - void useConfiguration(const Configuration & new_configuration); - - Configuration getConfigurationCopy() const; - - String getFormatCopy() const; - -private: - friend class StorageS3Cluster; - friend class TableFunctionS3Cluster; - friend class StorageS3Queue; - friend class ReadFromStorageS3Step; - - Configuration configuration; - mutable std::mutex configuration_update_mutex; - - String name; - const bool distributed_processing; - std::optional format_settings; - ASTPtr partition_by; - - static std::pair getTableStructureAndFormatFromDataImpl( - std::optional format, - const Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx); - - bool supportsSubcolumns() const override { return true; } - - bool supportsDynamicSubcolumns() const override { return true; } - - bool supportsSubsetOfColumns(const ContextPtr & context) const; - - bool prefersLargeBlocks() const override; - - bool parallelizeOutputAfterReading(ContextPtr context) const override; -}; - -std::unique_ptr createS3ReadBuffer( - const String & key, - size_t object_size, - std::shared_ptr context, - std::shared_ptr client_ptr, - const String & bucket, - const String & version_id, - const S3Settings::RequestSettings & request_settings); - -std::unique_ptr createAsyncS3ReadBuffer( - const String & key, - const ReadSettings & read_settings, - size_t object_size, - std::shared_ptr context, - std::shared_ptr client_ptr, - const String & bucket, - const String & version_id, - const S3Settings::RequestSettings & request_settings); -} - -#endif diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp deleted file mode 100644 index 0060450eea7..00000000000 --- a/src/Storages/StorageS3Cluster.cpp +++ /dev/null @@ -1,114 +0,0 @@ -#include "Storages/StorageS3Cluster.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -StorageS3Cluster::StorageS3Cluster( - const String & cluster_name_, - const StorageS3::Configuration & configuration_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const ContextPtr & context) - : IStorageCluster(cluster_name_, table_id_, getLogger("StorageS3Cluster (" + table_id_.table_name + ")")) - , s3_configuration{configuration_} -{ - context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri); - context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration_.headers_from_ast); - - StorageInMemoryMetadata storage_metadata; - updateConfigurationIfChanged(context); - - if (columns_.empty()) - { - ColumnsDescription columns; - /// `format_settings` is set to std::nullopt, because StorageS3Cluster is used only as table function - if (s3_configuration.format == "auto") - std::tie(columns, s3_configuration.format) = StorageS3::getTableStructureAndFormatFromData(s3_configuration, /*format_settings=*/std::nullopt, context); - else - columns = StorageS3::getTableStructureFromData(s3_configuration, /*format_settings=*/std::nullopt, context); - - storage_metadata.setColumns(columns); - } - else - { - if (s3_configuration.format == "auto") - s3_configuration.format = StorageS3::getTableStructureAndFormatFromData(s3_configuration, /*format_settings=*/std::nullopt, context).second; - - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - setInMemoryMetadata(storage_metadata); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); -} - -void StorageS3Cluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context) -{ - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query)); - - TableFunctionS3Cluster::updateStructureAndFormatArgumentsIfNeeded( - expression_list->children, - storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), - s3_configuration.format, - context); -} - -void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) -{ - s3_configuration.update(local_context); -} - -RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const -{ - auto iterator = std::make_shared( - *s3_configuration.client, - s3_configuration.url, - predicate, - getVirtualsList(), - context, - nullptr, - s3_configuration.request_settings, - context->getFileProgressCallback()); - - auto callback = std::make_shared>([iterator]() mutable -> String - { - if (auto next = iterator->next()) - return next->key; - return ""; - }); - return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; -} - -} - -#endif diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h deleted file mode 100644 index aa97bc01b02..00000000000 --- a/src/Storages/StorageS3Cluster.h +++ /dev/null @@ -1,53 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include - -namespace DB -{ - -class Context; - -class StorageS3Cluster : public IStorageCluster -{ -public: - StorageS3Cluster( - const String & cluster_name_, - const StorageS3::Configuration & configuration_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const ContextPtr & context_); - - std::string getName() const override { return "S3Cluster"; } - - RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; - - bool supportsSubcolumns() const override { return true; } - - bool supportsDynamicSubcolumns() const override { return true; } - - bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } - -protected: - void updateConfigurationIfChanged(ContextPtr local_context); - -private: - void updateBeforeRead(const ContextPtr & context) override { updateConfigurationIfChanged(context); } - - void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override; - - StorageS3::Configuration s3_configuration; -}; - - -} - -#endif diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 04634bcf1b3..b767805f637 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -18,18 +18,20 @@ namespace ErrorCodes extern const int INVALID_SETTING_VALUE; } -S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const Settings & settings) +S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const Settings & settings, bool validate_settings) { - updateFromSettingsImpl(settings, false); - validate(); + updateFromSettings(settings, false); + if (validate_settings) + validate(); } S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings, - String setting_name_prefix) - : PartUploadSettings(settings) + String setting_name_prefix, + bool validate_settings) + : PartUploadSettings(settings, validate_settings) { String key = config_prefix + "." + setting_name_prefix; strict_upload_part_size = config.getUInt64(key + "strict_upload_part_size", strict_upload_part_size); @@ -46,7 +48,8 @@ S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( storage_class_name = config.getString(config_prefix + ".s3_storage_class", storage_class_name); storage_class_name = Poco::toUpperInPlace(storage_class_name); - validate(); + if (validate_settings) + validate(); } S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedCollection & collection) @@ -65,7 +68,7 @@ S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedC validate(); } -void S3Settings::RequestSettings::PartUploadSettings::updateFromSettingsImpl(const Settings & settings, bool if_changed) +void S3Settings::RequestSettings::PartUploadSettings::updateFromSettings(const Settings & settings, bool if_changed) { if (!if_changed || settings.s3_strict_upload_part_size.changed) strict_upload_part_size = settings.s3_strict_upload_part_size; @@ -108,7 +111,7 @@ void S3Settings::RequestSettings::PartUploadSettings::validate() if (max_upload_part_size > max_upload_part_size_limit) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_upload_part_size has invalid value {} which is grater than the s3 API limit {}", + "Setting max_upload_part_size has invalid value {} which is greater than the s3 API limit {}", ReadableSize(max_upload_part_size), ReadableSize(max_upload_part_size_limit)); if (max_single_part_upload_size > max_upload_part_size_limit) @@ -170,8 +173,8 @@ void S3Settings::RequestSettings::PartUploadSettings::validate() } -S3Settings::RequestSettings::RequestSettings(const Settings & settings) - : upload_settings(settings) +S3Settings::RequestSettings::RequestSettings(const Settings & settings, bool validate_settings) + : upload_settings(settings, validate_settings) { updateFromSettingsImpl(settings, false); } @@ -190,8 +193,9 @@ S3Settings::RequestSettings::RequestSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings, - String setting_name_prefix) - : upload_settings(config, config_prefix, settings, setting_name_prefix) + String setting_name_prefix, + bool validate_settings) + : upload_settings(config, config_prefix, settings, setting_name_prefix, validate_settings) { String key = config_prefix + "." + setting_name_prefix; max_single_read_retries = config.getUInt64(key + "max_single_read_retries", settings.s3_max_single_read_retries); @@ -262,13 +266,12 @@ void S3Settings::RequestSettings::updateFromSettingsImpl(const Settings & settin request_timeout_ms = settings.s3_request_timeout_ms; } -void S3Settings::RequestSettings::updateFromSettings(const Settings & settings) +void S3Settings::RequestSettings::updateFromSettingsIfChanged(const Settings & settings) { updateFromSettingsImpl(settings, true); - upload_settings.updateFromSettings(settings); + upload_settings.updateFromSettings(settings, true); } - void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings) { std::lock_guard lock(mutex); @@ -292,7 +295,7 @@ void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::U } } -S3Settings StorageS3Settings::getSettings(const String & endpoint, const String & user, bool ignore_user) const +std::optional StorageS3Settings::getSettings(const String & endpoint, const String & user, bool ignore_user) const { std::lock_guard lock(mutex); auto next_prefix_setting = s3_settings.upper_bound(endpoint); diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 0f972db02b1..c3bc8aa6ed6 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -39,20 +39,19 @@ struct S3Settings size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; String storage_class_name; - void updateFromSettings(const Settings & settings) { updateFromSettingsImpl(settings, true); } + void updateFromSettings(const Settings & settings, bool if_changed); void validate(); private: PartUploadSettings() = default; - explicit PartUploadSettings(const Settings & settings); + explicit PartUploadSettings(const Settings & settings, bool validate_settings = true); explicit PartUploadSettings(const NamedCollection & collection); PartUploadSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings, - String setting_name_prefix = {}); - - void updateFromSettingsImpl(const Settings & settings, bool if_changed); + String setting_name_prefix = {}, + bool validate_settings = true); friend struct RequestSettings; }; @@ -80,7 +79,7 @@ struct S3Settings void setStorageClassName(const String & storage_class_name) { upload_settings.storage_class_name = storage_class_name; } RequestSettings() = default; - explicit RequestSettings(const Settings & settings); + explicit RequestSettings(const Settings & settings, bool validate_settings = true); explicit RequestSettings(const NamedCollection & collection); /// What's the setting_name_prefix, and why do we need it? @@ -94,9 +93,10 @@ struct S3Settings const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings, - String setting_name_prefix = {}); + String setting_name_prefix = {}, + bool validate_settings = true); - void updateFromSettings(const Settings & settings); + void updateFromSettingsIfChanged(const Settings & settings); private: void updateFromSettingsImpl(const Settings & settings, bool if_changed); @@ -112,7 +112,7 @@ class StorageS3Settings public: void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings); - S3Settings getSettings(const String & endpoint, const String & user, bool ignore_user = false) const; + std::optional getSettings(const String & endpoint, const String & user, bool ignore_user = false) const; private: mutable std::mutex mutex; diff --git a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp index 634089bd1cd..b67a8b23e9d 100644 --- a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp +++ b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp @@ -1,9 +1,7 @@ #include #include -#include #include -#include -#include +#include #include #include #include @@ -11,6 +9,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -76,14 +77,14 @@ void StorageSystemSchemaInferenceCache::fillData(MutableColumns & res_columns, C { fillDataImpl(res_columns, StorageFile::getSchemaCache(context), "File"); #if USE_AWS_S3 - fillDataImpl(res_columns, StorageS3::getSchemaCache(context), "S3"); + fillDataImpl(res_columns, StorageObjectStorage::getSchemaCache(context, StorageS3Configuration::type_name), "S3"); #endif #if USE_HDFS - fillDataImpl(res_columns, StorageHDFS::getSchemaCache(context), "HDFS"); + fillDataImpl(res_columns, StorageObjectStorage::getSchemaCache(context, StorageHDFSConfiguration::type_name), "HDFS"); #endif fillDataImpl(res_columns, StorageURL::getSchemaCache(context), "URL"); #if USE_AZURE_BLOB_STORAGE - fillDataImpl(res_columns, StorageAzureBlob::getSchemaCache(context), "Azure"); + fillDataImpl(res_columns, StorageObjectStorage::getSchemaCache(context, StorageAzureConfiguration::type_name), "Azure"); #endif } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index a9ec1f6c694..8bca1c97aad 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -297,7 +297,6 @@ namespace CASE_WINDOW_KIND(Year) #undef CASE_WINDOW_KIND } - UNREACHABLE(); } class AddingAggregatedChunkInfoTransform : public ISimpleTransform @@ -920,7 +919,6 @@ UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) CASE_WINDOW_KIND(Year) #undef CASE_WINDOW_KIND } - UNREACHABLE(); } UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) @@ -948,7 +946,6 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) CASE_WINDOW_KIND(Year) #undef CASE_WINDOW_KIND } - UNREACHABLE(); } void StorageWindowView::addFireSignal(std::set & signals) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index ce07c8e8d3e..0fb00c08acc 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -43,8 +43,6 @@ void registerStorageIceberg(StorageFactory & factory); #endif #if USE_HDFS -void registerStorageHDFS(StorageFactory & factory); - #if USE_HIVE void registerStorageHive(StorageFactory & factory); #endif @@ -97,9 +95,7 @@ void registerStorageSQLite(StorageFactory & factory); void registerStorageKeeperMap(StorageFactory & factory); -#if USE_AZURE_BLOB_STORAGE -void registerStorageAzureBlob(StorageFactory & factory); -#endif +void registerStorageObjectStorage(StorageFactory & factory); void registerStorages() { @@ -129,7 +125,6 @@ void registerStorages() #endif #if USE_AWS_S3 - registerStorageS3(factory); registerStorageHudi(factory); registerStorageS3Queue(factory); @@ -144,12 +139,9 @@ void registerStorages() #endif #if USE_HDFS - registerStorageHDFS(factory); - #if USE_HIVE registerStorageHive(factory); #endif - #endif registerStorageODBC(factory); @@ -197,9 +189,7 @@ void registerStorages() registerStorageKeeperMap(factory); - #if USE_AZURE_BLOB_STORAGE - registerStorageAzureBlob(factory); - #endif + registerStorageObjectStorage(factory); } } diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index 9f56d781bc9..28dc43f350b 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -1,13 +1,10 @@ #pragma once -#include "config.h" - #include #include #include #include -#include -#include +#include namespace DB diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 91165ba6705..fe6e5b3e593 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -1,15 +1,16 @@ #pragma once #include "config.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include -#if USE_AWS_S3 - -# include -# include -# include -# include -# include -# include namespace DB { @@ -23,44 +24,76 @@ public: protected: StoragePtr executeImpl( - const ASTPtr & /*ast_function*/, + const ASTPtr & /* ast_function */, ContextPtr context, const std::string & table_name, - ColumnsDescription /*cached_columns*/, + ColumnsDescription cached_columns, bool /*is_insert_query*/) const override { ColumnsDescription columns; - if (TableFunction::configuration.structure != "auto") - columns = parseColumnsListFromString(TableFunction::configuration.structure, context); + auto configuration = TableFunction::getConfiguration(); + if (configuration->structure != "auto") + columns = parseColumnsListFromString(configuration->structure, context); + else if (!cached_columns.empty()) + columns = cached_columns; StoragePtr storage = Storage::create( - TableFunction::configuration, context, LoadingStrictnessLevel::CREATE, StorageID(TableFunction::getDatabaseName(), table_name), - columns, ConstraintsDescription{}, String{}, std::nullopt); + configuration, context, StorageID(TableFunction::getDatabaseName(), table_name), + columns, ConstraintsDescription{}, String{}, std::nullopt, LoadingStrictnessLevel::CREATE); storage->startup(); return storage; } - const char * getStorageTypeName() const override { return Storage::name; } + const char * getStorageTypeName() const override { return name; } - ColumnsDescription getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const override + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override { - if (TableFunction::configuration.structure == "auto") + auto configuration = TableFunction::getConfiguration(); + if (configuration->structure == "auto") { context->checkAccess(TableFunction::getSourceAccessType()); - return Storage::getTableStructureFromData(TableFunction::configuration, std::nullopt, context); + auto object_storage = TableFunction::getObjectStorage(context, !is_insert_query); + return Storage::getTableStructureFromData(object_storage, configuration, std::nullopt, context); + } + else + { + return parseColumnsListFromString(configuration->structure, context); } - - return parseColumnsListFromString(TableFunction::configuration.structure, context); } void parseArguments(const ASTPtr & ast_function, ContextPtr context) override { + auto configuration = TableFunction::getConfiguration(); + configuration->format = "Parquet"; /// Set default format to Parquet if it's not specified in arguments. - TableFunction::configuration.format = "Parquet"; TableFunction::parseArguments(ast_function, context); } }; -} +struct TableFunctionIcebergName +{ + static constexpr auto name = "iceberg"; +}; + +struct TableFunctionDeltaLakeName +{ + static constexpr auto name = "deltaLake"; +}; + +struct TableFunctionHudiName +{ + static constexpr auto name = "hudi"; +}; + +#if USE_AWS_S3 +#if USE_AVRO +using TableFunctionIceberg = ITableFunctionDataLake; #endif +#if USE_PARQUET +using TableFunctionDeltaLake = ITableFunctionDataLake; +#endif +using TableFunctionHudi = ITableFunctionDataLake; +#endif + +} diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp deleted file mode 100644 index 7a17db2a1a8..00000000000 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ /dev/null @@ -1,395 +0,0 @@ -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "registerTableFunctions.h" -#include -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; -} - -namespace -{ - -bool isConnectionString(const std::string & candidate) -{ - return !candidate.starts_with("http"); -} - -} - -void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) -{ - /// Supported signatures: - /// - /// AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) - /// - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - { - StorageAzureBlob::processNamedCollectionResult(configuration, *named_collection); - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.blob_path).value_or("auto"); - } - else - { - if (engine_args.size() < 3 || engine_args.size() > 8) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage Azure requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); - - std::unordered_map engine_args_to_idx; - - configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); - configuration.is_connection_string = isConnectionString(configuration.connection_url); - - configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); - configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); - - auto is_format_arg - = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().exists(s); }; - - if (engine_args.size() == 4) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name/structure"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - } - else - { - configuration.structure = fourth_arg; - } - } - else if (engine_args.size() == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - } - } - else if (engine_args.size() == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - configuration.structure = checkAndGetLiteralArgument(engine_args[5], "structure"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name/structure"); - if (is_format_arg(sixth_arg)) - configuration.format = sixth_arg; - else - configuration.structure = sixth_arg; - } - } - else if (engine_args.size() == 7) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - } - else if (engine_args.size() == 8) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - configuration.structure = checkAndGetLiteralArgument(engine_args[7], "structure"); - } - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.blob_path).value_or("auto"); - } -} - -void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - /// Clone ast function, because we can modify its arguments like removing headers. - auto ast_copy = ast_function->clone(); - - ASTs & args_func = ast_function->children; - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); - - auto & args = args_func.at(0)->children; - - parseArgumentsImpl(args, context); -} - -void TableFunctionAzureBlobStorage::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context) -{ - if (auto collection = tryGetNamedCollectionWithOverrides(args, context)) - { - /// In case of named collection, just add key-value pairs "format='...', structure='...'" - /// at the end of arguments to override existed format and structure with "auto" values. - if (collection->getOrDefault("format", "auto") == "auto") - { - ASTs format_equal_func_args = {std::make_shared("format"), std::make_shared(format)}; - auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args)); - args.push_back(format_equal_func); - } - if (collection->getOrDefault("structure", "auto") == "auto") - { - ASTs structure_equal_func_args = {std::make_shared("structure"), std::make_shared(structure)}; - auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args)); - args.push_back(structure_equal_func); - } - } - else - { - if (args.size() < 3 || args.size() > 8) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage Azure requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); - - auto format_literal = std::make_shared(format); - auto structure_literal = std::make_shared(structure); - - for (auto & arg : args) - arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); - - auto is_format_arg - = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().exists(s); }; - - /// (connection_string, container_name, blobpath) - if (args.size() == 3) - { - args.push_back(format_literal); - /// Add compression = "auto" before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - /// (connection_string, container_name, blobpath, structure) or - /// (connection_string, container_name, blobpath, format) - /// We can distinguish them by looking at the 4-th argument: check if it's format name or not. - else if (args.size() == 4) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name/structure"); - /// (..., format) -> (..., format, compression, structure) - if (is_format_arg(fourth_arg)) - { - if (fourth_arg == "auto") - args[3] = format_literal; - /// Add compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - /// (..., structure) -> (..., format, compression, structure) - else - { - auto structure_arg = args.back(); - args[3] = format_literal; - /// Add compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - if (fourth_arg == "auto") - args.push_back(structure_literal); - else - args.push_back(structure_arg); - } - } - /// (connection_string, container_name, blobpath, format, compression) or - /// (storage_account_url, container_name, blobpath, account_name, account_key) - /// We can distinguish them by looking at the 4-th argument: check if it's format name or not. - else if (args.size() == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); - /// (..., format, compression) -> (..., format, compression, structure) - if (is_format_arg(fourth_arg)) - { - if (fourth_arg == "auto") - args[3] = format_literal; - args.push_back(structure_literal); - } - /// (..., account_name, account_key) -> (..., account_name, account_key, format, compression, structure) - else - { - args.push_back(format_literal); - /// Add compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - } - /// (connection_string, container_name, blobpath, format, compression, structure) or - /// (storage_account_url, container_name, blobpath, account_name, account_key, structure) or - /// (storage_account_url, container_name, blobpath, account_name, account_key, format) - else if (args.size() == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); - auto sixth_arg = checkAndGetLiteralArgument(args[5], "format/structure"); - - /// (..., format, compression, structure) - if (is_format_arg(fourth_arg)) - { - if (fourth_arg == "auto") - args[3] = format_literal; - if (checkAndGetLiteralArgument(args[5], "structure") == "auto") - args[5] = structure_literal; - } - /// (..., account_name, account_key, format) -> (..., account_name, account_key, format, compression, structure) - else if (is_format_arg(sixth_arg)) - { - if (sixth_arg == "auto") - args[5] = format_literal; - /// Add compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - /// (..., account_name, account_key, structure) -> (..., account_name, account_key, format, compression, structure) - else - { - auto structure_arg = args.back(); - args[5] = format_literal; - /// Add compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - if (sixth_arg == "auto") - args.push_back(structure_literal); - else - args.push_back(structure_arg); - } - } - /// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression) - else if (args.size() == 7) - { - /// (..., format, compression) -> (..., format, compression, structure) - if (checkAndGetLiteralArgument(args[5], "format") == "auto") - args[5] = format_literal; - args.push_back(structure_literal); - } - /// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) - else if (args.size() == 8) - { - if (checkAndGetLiteralArgument(args[5], "format") == "auto") - args[5] = format_literal; - if (checkAndGetLiteralArgument(args[7], "structure") == "auto") - args[7] = structure_literal; - } - } -} - -ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(ContextPtr context, bool is_insert_query) const -{ - if (configuration.structure == "auto") - { - context->checkAccess(getSourceAccessType()); - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); - - auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()); - if (configuration.format == "auto") - return StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, std::nullopt, context).first; - return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); - } - - return parseColumnsListFromString(configuration.structure, context); -} - -bool TableFunctionAzureBlobStorage::supportsReadingSubsetOfColumns(const ContextPtr & context) -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context); -} - -std::unordered_set TableFunctionAzureBlobStorage::getVirtualsToCheckBeforeUsingStructureHint() const -{ - return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); -} - -StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const -{ - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); - - ColumnsDescription columns; - if (configuration.structure != "auto") - columns = parseColumnsListFromString(configuration.structure, context); - else if (!structure_hint.empty()) - columns = structure_hint; - - StoragePtr storage = std::make_shared( - configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()), - context, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - String{}, - /// No format_settings for table function Azure - std::nullopt, - /* distributed_processing */ false, - nullptr); - - storage->startup(); - - return storage; -} - -void registerTableFunctionAzureBlobStorage(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description=R"(The table function can be used to read the data stored on Azure Blob Storage.)", - .examples{{"azureBlobStorage", "SELECT * FROM azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, - .allow_readonly = false}); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h deleted file mode 100644 index 9622881b417..00000000000 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ /dev/null @@ -1,80 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include - - -namespace DB -{ - -class Context; - -/* AzureBlob(source, [access_key_id, secret_access_key,] [format, compression, structure]) - creates a temporary storage for a file in AzureBlob. - */ -class TableFunctionAzureBlobStorage : public ITableFunction -{ -public: - static constexpr auto name = "azureBlobStorage"; - - static constexpr auto signature = " - connection_string, container_name, blobpath\n" - " - connection_string, container_name, blobpath, structure \n" - " - connection_string, container_name, blobpath, format \n" - " - connection_string, container_name, blobpath, format, compression \n" - " - connection_string, container_name, blobpath, format, compression, structure \n" - " - storage_account_url, container_name, blobpath, account_name, account_key\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n"; - - static size_t getMaxNumberOfArguments() { return 8; } - - String getName() const override - { - return name; - } - - virtual String getSignature() const - { - return signature; - } - - bool hasStaticStructure() const override { return configuration.structure != "auto"; } - - bool needStructureHint() const override { return configuration.structure == "auto"; } - - void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } - - bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; - - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; - - virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); - - static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context); - -protected: - - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "Azure"; } - - ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - - mutable StorageAzureBlob::Configuration configuration; - ColumnsDescription structure_hint; -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp deleted file mode 100644 index 02b24dccf86..00000000000 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ /dev/null @@ -1,83 +0,0 @@ -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include - -#include "registerTableFunctions.h" - -#include - - -namespace DB -{ - -StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( - const ASTPtr & /*function*/, ContextPtr context, - const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const -{ - StoragePtr storage; - ColumnsDescription columns; - - if (configuration.structure != "auto") - { - columns = parseColumnsListFromString(configuration.structure, context); - } - else if (!structure_hint.empty()) - { - columns = structure_hint; - } - - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); - - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) - { - /// On worker node this filename won't contains globs - storage = std::make_shared( - configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()), - context, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - /* comment */String{}, - /* format_settings */std::nullopt, /// No format_settings - /* distributed_processing */ true, - /*partition_by_=*/nullptr); - } - else - { - storage = std::make_shared( - cluster_name, - configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container, configuration.getConnectionURL().toString()), - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - context); - } - - storage->startup(); - - return storage; -} - - -void registerTableFunctionAzureBlobStorageCluster(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description=R"(The table function can be used to read the data stored on Azure Blob Storage in parallel for many nodes in a specified cluster.)", - .examples{{"azureBlobStorageCluster", "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, - .allow_readonly = false} - ); -} - - -} - -#endif diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h deleted file mode 100644 index 58f79328f63..00000000000 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h +++ /dev/null @@ -1,55 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include - - -namespace DB -{ - -class Context; - -/** - * azureBlobStorageCluster(cluster_name, source, [access_key_id, secret_access_key,] format, compression_method, structure) - * A table function, which allows to process many files from Azure Blob Storage on a specific cluster - * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks - * in Azure Blob Storage file path and dispatch each file dynamically. - * On worker node it asks initiator about next task to process, processes it. - * This is repeated until the tasks are finished. - */ -class TableFunctionAzureBlobStorageCluster : public ITableFunctionCluster -{ -public: - static constexpr auto name = "azureBlobStorageCluster"; - static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]"; - - String getName() const override - { - return name; - } - - String getSignature() const override - { - return signature; - } - -protected: - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "AzureBlobStorageCluster"; } -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionDeltaLake.cpp b/src/TableFunctions/TableFunctionDeltaLake.cpp deleted file mode 100644 index b8bf810f6fa..00000000000 --- a/src/TableFunctions/TableFunctionDeltaLake.cpp +++ /dev/null @@ -1,33 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 && USE_PARQUET - -#include -#include -#include -#include -#include "registerTableFunctions.h" - -namespace DB -{ - -struct TableFunctionDeltaLakeName -{ - static constexpr auto name = "deltaLake"; -}; - -using TableFunctionDeltaLake = ITableFunctionDataLake; - -void registerTableFunctionDeltaLake(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation = { - .description=R"(The table function can be used to read the DeltaLake table stored on object store.)", - .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp deleted file mode 100644 index 45829245551..00000000000 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ /dev/null @@ -1,50 +0,0 @@ -#include "config.h" -#include "registerTableFunctions.h" - -#if USE_HDFS -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -StoragePtr TableFunctionHDFS::getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, - const std::string & table_name, const String & compression_method_) const -{ - return std::make_shared( - source, - StorageID(getDatabaseName(), table_name), - format_, - columns, - ConstraintsDescription{}, - String{}, - global_context, - compression_method_); -} - -ColumnsDescription TableFunctionHDFS::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const -{ - if (structure == "auto") - { - context->checkAccess(getSourceAccessType()); - if (format == "auto") - return StorageHDFS::getTableStructureAndFormatFromData(filename, compression_method, context).first; - return StorageHDFS::getTableStructureFromData(format, filename, compression_method, context); - } - - return parseColumnsListFromString(structure, context); -} - -void registerTableFunctionHDFS(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} -#endif diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h deleted file mode 100644 index f1c0b8a7eae..00000000000 --- a/src/TableFunctions/TableFunctionHDFS.h +++ /dev/null @@ -1,48 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_HDFS - -#include - - -namespace DB -{ - -class Context; - -/* hdfs(URI, [format, structure, compression]) - creates a temporary storage from hdfs files - * - */ -class TableFunctionHDFS : public ITableFunctionFileLike -{ -public: - static constexpr auto name = "hdfs"; - static constexpr auto signature = " - uri\n" - " - uri, format\n" - " - uri, format, structure\n" - " - uri, format, structure, compression_method\n"; - - String getName() const override - { - return name; - } - - String getSignature() const override - { - return signature; - } - - ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - -private: - StoragePtr getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, - const std::string & table_name, const String & compression_method_) const override; - const char * getStorageTypeName() const override { return "HDFS"; } -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp deleted file mode 100644 index 57ce6d2b9ff..00000000000 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ /dev/null @@ -1,60 +0,0 @@ -#include "config.h" - -#if USE_HDFS - -#include -#include - -#include -#include -#include "registerTableFunctions.h" - -#include - - -namespace DB -{ - -StoragePtr TableFunctionHDFSCluster::getStorage( - const String & /*source*/, const String & /*format_*/, const ColumnsDescription & columns, ContextPtr context, - const std::string & table_name, const String & /*compression_method_*/) const -{ - StoragePtr storage; - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) - { - /// On worker node this uri won't contains globs - storage = std::make_shared( - filename, - StorageID(getDatabaseName(), table_name), - format, - columns, - ConstraintsDescription{}, - String{}, - context, - compression_method, - /*distributed_processing=*/true, - nullptr); - } - else - { - storage = std::make_shared( - context, - cluster_name, - filename, - StorageID(getDatabaseName(), table_name), - format, - columns, - ConstraintsDescription{}, - compression_method); - } - return storage; -} - -void registerTableFunctionHDFSCluster(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionHDFSCluster.h b/src/TableFunctions/TableFunctionHDFSCluster.h deleted file mode 100644 index 0253217feb7..00000000000 --- a/src/TableFunctions/TableFunctionHDFSCluster.h +++ /dev/null @@ -1,54 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_HDFS - -#include -#include -#include - - -namespace DB -{ - -class Context; - -/** - * hdfsCluster(cluster, URI, format, structure, compression_method) - * A table function, which allows to process many files from HDFS on a specific cluster - * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks - * in HDFS file path and dispatch each file dynamically. - * On worker node it asks initiator about next task to process, processes it. - * This is repeated until the tasks are finished. - */ -class TableFunctionHDFSCluster : public ITableFunctionCluster -{ -public: - static constexpr auto name = "hdfsCluster"; - static constexpr auto signature = " - cluster_name, uri\n" - " - cluster_name, uri, format\n" - " - cluster_name, uri, format, structure\n" - " - cluster_name, uri, format, structure, compression_method\n"; - - String getName() const override - { - return name; - } - - String getSignature() const override - { - return signature; - } - -protected: - StoragePtr getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, - const std::string & table_name, const String & compression_method_) const override; - - const char * getStorageTypeName() const override { return "HDFSCluster"; } -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp deleted file mode 100644 index 436e708b72d..00000000000 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ /dev/null @@ -1,31 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include "registerTableFunctions.h" - -namespace DB -{ - -struct TableFunctionHudiName -{ - static constexpr auto name = "hudi"; -}; -using TableFunctionHudi = ITableFunctionDataLake; - -void registerTableFunctionHudi(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description=R"(The table function can be used to read the Hudi table stored on object store.)", - .examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} -} - -#endif diff --git a/src/TableFunctions/TableFunctionIceberg.cpp b/src/TableFunctions/TableFunctionIceberg.cpp deleted file mode 100644 index d37aace01c6..00000000000 --- a/src/TableFunctions/TableFunctionIceberg.cpp +++ /dev/null @@ -1,34 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 && USE_AVRO - -#include -#include -#include -#include -#include "registerTableFunctions.h" - - -namespace DB -{ - -struct TableFunctionIcebergName -{ - static constexpr auto name = "iceberg"; -}; - -using TableFunctionIceberg = ITableFunctionDataLake; - -void registerTableFunctionIceberg(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description=R"(The table function can be used to read the Iceberg table stored on object store.)", - .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp new file mode 100644 index 00000000000..550d9cc799b --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -0,0 +1,226 @@ +#include "config.h" + +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +template +ObjectStoragePtr TableFunctionObjectStorage::getObjectStorage(const ContextPtr & context, bool create_readonly) const +{ + if (!object_storage) + object_storage = configuration->createObjectStorage(context, create_readonly); + return object_storage; +} + +template +StorageObjectStorage::ConfigurationPtr TableFunctionObjectStorage::getConfiguration() const +{ + if (!configuration) + configuration = std::make_shared(); + return configuration; +} + +template +std::vector TableFunctionObjectStorage::skipAnalysisForArguments( + const QueryTreeNodePtr & query_node_table_function, ContextPtr) const +{ + auto & table_function_node = query_node_table_function->as(); + auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); + size_t table_function_arguments_size = table_function_arguments_nodes.size(); + + std::vector result; + for (size_t i = 0; i < table_function_arguments_size; ++i) + { + auto * function_node = table_function_arguments_nodes[i]->as(); + if (function_node && function_node->getFunctionName() == "headers") + result.push_back(i); + } + return result; +} + +template +void TableFunctionObjectStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + /// Clone ast function, because we can modify its arguments like removing headers. + auto ast_copy = ast_function->clone(); + ASTs & args_func = ast_copy->children; + if (args_func.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); + + auto & args = args_func.at(0)->children; + parseArgumentsImpl(args, context); +} + +template +ColumnsDescription TableFunctionObjectStorage< + Definition, Configuration>::getActualTableStructure(ContextPtr context, bool is_insert_query) const +{ + if (configuration->structure == "auto") + { + context->checkAccess(getSourceAccessType()); + ColumnsDescription columns; + auto storage = getObjectStorage(context, !is_insert_query); + resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, context); + return columns; + } + else + return parseColumnsListFromString(configuration->structure, context); +} + +template +StoragePtr TableFunctionObjectStorage::executeImpl( + const ASTPtr & /* ast_function */, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const +{ + ColumnsDescription columns; + chassert(configuration); + if (configuration->structure != "auto") + columns = parseColumnsListFromString(configuration->structure, context); + else if (!structure_hint.empty()) + columns = structure_hint; + else if (!cached_columns.empty()) + columns = cached_columns; + + StoragePtr storage = std::make_shared( + configuration, + getObjectStorage(context, !is_insert_query), + context, + StorageID(getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + String{}, + /* format_settings */std::nullopt, + /* distributed_processing */false, + nullptr); + + storage->startup(); + return storage; +} + +void registerTableFunctionObjectStorage(TableFunctionFactory & factory) +{ + UNUSED(factory); +#if USE_AWS_S3 + factory.registerFunction>( + { + .documentation = + { + .description=R"(The table function can be used to read the data stored on AWS S3.)", + .examples{{"s3", "SELECT * FROM s3(url, access_key_id, secret_access_key)", ""} + }, + .categories{"DataLake"}}, + .allow_readonly = false + }); + + factory.registerFunction>( + { + .documentation = + { + .description=R"(The table function can be used to read the data stored on GCS.)", + .examples{{"gcs", "SELECT * FROM gcs(url, access_key_id, secret_access_key)", ""} + }, + .categories{"DataLake"}}, + .allow_readonly = false + }); + + factory.registerFunction>( + { + .documentation = + { + .description=R"(The table function can be used to read the data stored on COSN.)", + .examples{{"cosn", "SELECT * FROM cosn(url, access_key_id, secret_access_key)", ""} + }, + .categories{"DataLake"}}, + .allow_readonly = false + }); + factory.registerFunction>( + { + .documentation = + { + .description=R"(The table function can be used to read the data stored on OSS.)", + .examples{{"oss", "SELECT * FROM oss(url, access_key_id, secret_access_key)", ""} + }, + .categories{"DataLake"}}, + .allow_readonly = false + }); +#endif + +#if USE_AZURE_BLOB_STORAGE + factory.registerFunction>( + { + .documentation = + { + .description=R"(The table function can be used to read the data stored on Azure Blob Storage.)", + .examples{ + { + "azureBlobStorage", + "SELECT * FROM azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure])", "" + }} + }, + .allow_readonly = false + }); +#endif +#if USE_HDFS + factory.registerFunction>( + { + .documentation = + { + .description=R"(The table function can be used to read the data stored on HDFS virtual filesystem.)", + .examples{ + { + "hdfs", + "SELECT * FROM hdfs(url, format, compression, structure])", "" + }} + }, + .allow_readonly = false + }); +#endif +} + +#if USE_AZURE_BLOB_STORAGE +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +#endif + +#if USE_AWS_S3 +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +#endif + +#if USE_HDFS +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +#endif + +} diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h new file mode 100644 index 00000000000..86b8f0d5e14 --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -0,0 +1,172 @@ +#pragma once + +#include "config.h" +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; +class StorageS3Configuration; +class StorageAzureConfiguration; +class StorageHDFSConfiguration; +struct S3StorageSettings; +struct AzureStorageSettings; +struct HDFSStorageSettings; + +struct AzureDefinition +{ + static constexpr auto name = "azureBlobStorage"; + static constexpr auto storage_type_name = "Azure"; + static constexpr auto signature = " - connection_string, container_name, blobpath\n" + " - connection_string, container_name, blobpath, structure \n" + " - connection_string, container_name, blobpath, format \n" + " - connection_string, container_name, blobpath, format, compression \n" + " - connection_string, container_name, blobpath, format, compression, structure \n" + " - storage_account_url, container_name, blobpath, account_name, account_key\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n"; + static constexpr auto max_number_of_arguments = 8; +}; + +struct S3Definition +{ + static constexpr auto name = "s3"; + static constexpr auto storage_type_name = "S3"; + static constexpr auto signature = " - url\n" + " - url, format\n" + " - url, format, structure\n" + " - url, format, structure, compression_method\n" + " - url, access_key_id, secret_access_key\n" + " - url, access_key_id, secret_access_key, session_token\n" + " - url, access_key_id, secret_access_key, format\n" + " - url, access_key_id, secret_access_key, session_token, format\n" + " - url, access_key_id, secret_access_key, format, structure\n" + " - url, access_key_id, secret_access_key, session_token, format, structure\n" + " - url, access_key_id, secret_access_key, format, structure, compression_method\n" + " - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + static constexpr auto max_number_of_arguments = 8; +}; + +struct GCSDefinition +{ + static constexpr auto name = "gcs"; + static constexpr auto storage_type_name = "GCS"; + static constexpr auto signature = S3Definition::signature; + static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; +}; + +struct COSNDefinition +{ + static constexpr auto name = "cosn"; + static constexpr auto storage_type_name = "COSN"; + static constexpr auto signature = S3Definition::signature; + static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; +}; + +struct OSSDefinition +{ + static constexpr auto name = "oss"; + static constexpr auto storage_type_name = "OSS"; + static constexpr auto signature = S3Definition::signature; + static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; +}; + +struct HDFSDefinition +{ + static constexpr auto name = "hdfs"; + static constexpr auto storage_type_name = "HDFS"; + static constexpr auto signature = " - uri\n" + " - uri, format\n" + " - uri, format, structure\n" + " - uri, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = 4; +}; + +template +class TableFunctionObjectStorage : public ITableFunction +{ +public: + static constexpr auto name = Definition::name; + static constexpr auto signature = Definition::signature; + + static size_t getMaxNumberOfArguments() { return Definition::max_number_of_arguments; } + + String getName() const override { return name; } + + virtual String getSignature() const { return signature; } + + bool hasStaticStructure() const override { return configuration->structure != "auto"; } + + bool needStructureHint() const override { return configuration->structure == "auto"; } + + void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } + + bool supportsReadingSubsetOfColumns(const ContextPtr & context) override + { + return configuration->format != "auto" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); + } + + std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override + { + return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); + } + + virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context) + { + StorageObjectStorage::Configuration::initialize(*getConfiguration(), args, context, true); + } + + static void updateStructureAndFormatArgumentsIfNeeded( + ASTs & args, + const String & structure, + const String & format, + const ContextPtr & context) + { + Configuration().addStructureAndFormatToArgs(args, structure, format, context); + } + +protected: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + const char * getStorageTypeName() const override { return Definition::storage_type_name; } + + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + ObjectStoragePtr getObjectStorage(const ContextPtr & context, bool create_readonly) const; + ConfigurationPtr getConfiguration() const; + + mutable ConfigurationPtr configuration; + mutable ObjectStoragePtr object_storage; + ColumnsDescription structure_hint; + + std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; +}; + +#if USE_AWS_S3 +using TableFunctionS3 = TableFunctionObjectStorage; +#endif + +#if USE_AZURE_BLOB_STORAGE +using TableFunctionAzureBlob = TableFunctionObjectStorage; +#endif + +#if USE_HDFS +using TableFunctionHDFS = TableFunctionObjectStorage; +#endif +} diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp new file mode 100644 index 00000000000..449bd2c8c49 --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -0,0 +1,118 @@ +#include "config.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +template +StoragePtr TableFunctionObjectStorageCluster::executeImpl( + const ASTPtr & /*function*/, ContextPtr context, + const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const +{ + auto configuration = Base::getConfiguration(); + + ColumnsDescription columns; + if (configuration->structure != "auto") + columns = parseColumnsListFromString(configuration->structure, context); + else if (!Base::structure_hint.empty()) + columns = Base::structure_hint; + else if (!cached_columns.empty()) + columns = cached_columns; + + auto object_storage = Base::getObjectStorage(context, !is_insert_query); + StoragePtr storage; + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) + { + /// On worker node this filename won't contains globs + storage = std::make_shared( + configuration, + object_storage, + context, + StorageID(Base::getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + /* comment */String{}, + /* format_settings */std::nullopt, /// No format_settings + /* distributed_processing */true, + /*partition_by_=*/nullptr); + } + else + { + storage = std::make_shared( + ITableFunctionCluster::cluster_name, + configuration, + object_storage, + StorageID(Base::getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + context); + } + + storage->startup(); + return storage; +} + + +void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory) +{ +#if USE_AWS_S3 + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the data stored on S3 in parallel for many nodes in a specified cluster.)", + .examples{{"s3Cluster", "SELECT * FROM s3Cluster(cluster, url, format, structure)", ""}}}, + .allow_readonly = false + } + ); +#endif + +#if USE_AZURE_BLOB_STORAGE + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the data stored on Azure Blob Storage in parallel for many nodes in a specified cluster.)", + .examples{{ + "azureBlobStorageCluster", + "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure])", ""}}}, + .allow_readonly = false + } + ); +#endif + +#if USE_HDFS + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the data stored on HDFS in parallel for many nodes in a specified cluster.)", + .examples{{"HDFSCluster", "SELECT * FROM HDFSCluster(cluster_name, uri, format)", ""}}}, + .allow_readonly = false + } + ); +#endif + + UNUSED(factory); +} + +#if USE_AWS_S3 +template class TableFunctionObjectStorageCluster; +#endif + +#if USE_AZURE_BLOB_STORAGE +template class TableFunctionObjectStorageCluster; +#endif + +#if USE_HDFS +template class TableFunctionObjectStorageCluster; +#endif +} diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.h b/src/TableFunctions/TableFunctionObjectStorageCluster.h new file mode 100644 index 00000000000..296791b8bda --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.h @@ -0,0 +1,102 @@ +#pragma once +#include "config.h" +#include +#include +#include + + +namespace DB +{ + +class Context; + +class StorageS3Settings; +class StorageAzureBlobSettings; +class StorageS3Configuration; +class StorageAzureConfiguration; + +struct AzureClusterDefinition +{ + static constexpr auto name = "azureBlobStorageCluster"; + static constexpr auto storage_type_name = "AzureBlobStorageCluster"; + static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]"; + static constexpr auto max_number_of_arguments = AzureDefinition::max_number_of_arguments + 1; +}; + +struct S3ClusterDefinition +{ + static constexpr auto name = "s3Cluster"; + static constexpr auto storage_type_name = "S3Cluster"; + static constexpr auto signature = " - cluster, url\n" + " - cluster, url, format\n" + " - cluster, url, format, structure\n" + " - cluster, url, access_key_id, secret_access_key\n" + " - cluster, url, format, structure, compression_method\n" + " - cluster, url, access_key_id, secret_access_key, format\n" + " - cluster, url, access_key_id, secret_access_key, format, structure\n" + " - cluster, url, access_key_id, secret_access_key, format, structure, compression_method\n" + " - cluster, url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments + 1; +}; + +struct HDFSClusterDefinition +{ + static constexpr auto name = "hdfsCluster"; + static constexpr auto storage_type_name = "HDFSCluster"; + static constexpr auto signature = " - cluster_name, uri\n" + " - cluster_name, uri, format\n" + " - cluster_name, uri, format, structure\n" + " - cluster_name, uri, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = HDFSDefinition::max_number_of_arguments + 1; +}; + +/** +* Class implementing s3/hdfs/azureBlobStorage)Cluster(...) table functions, +* which allow to process many files from S3/HDFS/Azure blob storage on a specific cluster. +* On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks +* in file path and dispatch each file dynamically. +* On worker node it asks initiator about next task to process, processes it. +* This is repeated until the tasks are finished. +*/ +template +class TableFunctionObjectStorageCluster : public ITableFunctionCluster> +{ +public: + static constexpr auto name = Definition::name; + static constexpr auto signature = Definition::signature; + + String getName() const override { return name; } + String getSignature() const override { return signature; } + +protected: + using Base = TableFunctionObjectStorage; + + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + const char * getStorageTypeName() const override { return Definition::storage_type_name; } + + bool hasStaticStructure() const override { return Base::getConfiguration()->structure != "auto"; } + + bool needStructureHint() const override { return Base::getConfiguration()->structure == "auto"; } + + void setStructureHint(const ColumnsDescription & structure_hint_) override { Base::structure_hint = structure_hint_; } +}; + +#if USE_AWS_S3 +using TableFunctionS3Cluster = TableFunctionObjectStorageCluster; +#endif + +#if USE_AZURE_BLOB_STORAGE +using TableFunctionAzureBlobCluster = TableFunctionObjectStorageCluster; +#endif + +#if USE_HDFS +using TableFunctionHDFSCluster = TableFunctionObjectStorageCluster; +#endif +} diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp deleted file mode 100644 index dfb427a3bba..00000000000 --- a/src/TableFunctions/TableFunctionS3.cpp +++ /dev/null @@ -1,518 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "registerTableFunctions.h" -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int LOGICAL_ERROR; -} - - -std::vector TableFunctionS3::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const -{ - auto & table_function_node = query_node_table_function->as(); - auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); - size_t table_function_arguments_size = table_function_arguments_nodes.size(); - - std::vector result; - - for (size_t i = 0; i < table_function_arguments_size; ++i) - { - auto * function_node = table_function_arguments_nodes[i]->as(); - if (function_node && function_node->getFunctionName() == "headers") - result.push_back(i); - } - - return result; -} - -/// This is needed to avoid copy-paste. Because s3Cluster arguments only differ in additional argument (first) - cluster name -void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context) -{ - if (auto named_collection = tryGetNamedCollectionWithOverrides(args, context)) - { - StorageS3::processNamedCollectionResult(configuration, *named_collection); - if (configuration.format == "auto") - { - String file_path = named_collection->getOrDefault("filename", Poco::URI(named_collection->get("url")).getPath()); - configuration.format = FormatFactory::instance().tryGetFormatFromFileName(file_path).value_or("auto"); - } - } - else - { - size_t count = StorageURL::evalArgsAndCollectHeaders(args, configuration.headers_from_ast, context); - - if (count == 0 || count > 7) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); - - std::unordered_map args_to_idx; - - bool no_sign_request = false; - - /// For 2 arguments we support 2 possible variants: - /// - s3(source, format) - /// - s3(source, NOSIGN) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. - if (count == 2) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - no_sign_request = true; - else - args_to_idx = {{"format", 1}}; - } - /// For 3 arguments we support 3 possible variants: - /// - s3(source, format, structure) - /// - s3(source, access_key_id, secret_access_key) - /// - s3(source, NOSIGN, format) - /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. - else if (count == 3) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - args_to_idx = {{"format", 2}}; - } - else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) - args_to_idx = {{"format", 1}, {"structure", 2}}; - else - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; - } - /// For 4 arguments we support 4 possible variants: - /// - s3(source, format, structure, compression_method), - /// - s3(source, access_key_id, secret_access_key, format), - /// - s3(source, access_key_id, secret_access_key, session_token) - /// - s3(source, NOSIGN, format, structure) - /// We can distinguish them by looking at the 2-nd and 4-th argument: check if it's a format name or not. - else if (count == 4) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - args_to_idx = {{"format", 2}, {"structure", 3}}; - } - else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) - { - args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; - } - else - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); - if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; - } - else - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}}; - } - } - } - /// For 5 arguments we support 3 possible variants: - /// - s3(source, access_key_id, secret_access_key, format, structure) - /// - s3(source, access_key_id, secret_access_key, session_token, format) - /// - s3(source, NOSIGN, format, structure, compression_method) - /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or no, - /// and by the 4-th argument, check if it's a format name or not - else if (count == 5) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "NOSIGN/access_key_id"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - args_to_idx = {{"format", 2}, {"structure", 3}, {"compression_method", 4}}; - } - else - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); - if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}; - } - else - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}}; - } - } - } - // For 6 arguments we support 2 possible variants: - /// - s3(source, access_key_id, secret_access_key, format, structure, compression_method) - /// - s3(source, access_key_id, secret_access_key, session_token, format, structure) - /// We can distinguish them by looking at the 4-th argument: check if it's a format name or not - else if (count == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); - if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}; - } - else - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}}; - } - } - else if (count == 7) - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}; - } - - /// This argument is always the first - String url = checkAndGetLiteralArgument(args[0], "url"); - configuration.url = S3::URI(url); - - if (args_to_idx.contains("format")) - { - auto format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); - /// Set format to configuration only of it's not 'auto', - /// because we can have default format set in configuration. - if (format != "auto") - configuration.format = format; - } - - if (args_to_idx.contains("structure")) - configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); - - if (args_to_idx.contains("compression_method")) - configuration.compression_method = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); - - if (args_to_idx.contains("access_key_id")) - configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); - - if (args_to_idx.contains("secret_access_key")) - configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); - - if (args_to_idx.contains("session_token")) - configuration.auth_settings.session_token = checkAndGetLiteralArgument(args[args_to_idx["session_token"]], "session_token"); - - configuration.auth_settings.no_sign_request = no_sign_request; - - if (configuration.format == "auto") - { - if (configuration.url.archive_pattern.has_value()) - { - configuration.format = FormatFactory::instance() - .tryGetFormatFromFileName(Poco::URI(configuration.url.archive_pattern.value()).getPath()) - .value_or("auto"); - } - else - { - configuration.format - = FormatFactory::instance().tryGetFormatFromFileName(Poco::URI(configuration.url.uri_str).getPath()).value_or("auto"); - } - } - } - - configuration.keys = {configuration.url.key}; -} - -void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - /// Clone ast function, because we can modify its arguments like removing headers. - auto ast_copy = ast_function->clone(); - - /// Parse args - ASTs & args_func = ast_function->children; - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); - - auto & args = args_func.at(0)->children; - - parseArgumentsImpl(args, context); -} - -void TableFunctionS3::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context) -{ - if (auto collection = tryGetNamedCollectionWithOverrides(args, context)) - { - /// In case of named collection, just add key-value pairs "format='...', structure='...'" - /// at the end of arguments to override existed format and structure with "auto" values. - if (collection->getOrDefault("format", "auto") == "auto") - { - ASTs format_equal_func_args = {std::make_shared("format"), std::make_shared(format)}; - auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args)); - args.push_back(format_equal_func); - } - if (collection->getOrDefault("structure", "auto") == "auto") - { - ASTs structure_equal_func_args = {std::make_shared("structure"), std::make_shared(structure)}; - auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args)); - args.push_back(structure_equal_func); - } - } - else - { - HTTPHeaderEntries tmp_headers; - size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context); - - if (count == 0 || count > getMaxNumberOfArguments()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), count); - - auto format_literal = std::make_shared(format); - auto structure_literal = std::make_shared(structure); - - /// s3(s3_url) -> s3(s3_url, format, structure) - if (count == 1) - { - args.push_back(format_literal); - args.push_back(structure_literal); - } - /// s3(s3_url, format) -> s3(s3_url, format, structure) or - /// s3(s3_url, NOSIGN) -> s3(s3_url, NOSIGN, format, structure) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. - else if (count == 2) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - args.push_back(format_literal); - else if (second_arg == "auto") - args.back() = format_literal; - args.push_back(structure_literal); - } - /// s3(source, format, structure) or - /// s3(source, access_key_id, secret_access_key) or - /// s3(source, NOSIGN, format) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. - else if (count == 3) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - /// s3(source, NOSIGN, format) -> s3(source, NOSIGN, format, structure) - if (boost::iequals(second_arg, "NOSIGN")) - { - if (checkAndGetLiteralArgument(args[2], "format") == "auto") - args.back() = format_literal; - args.push_back(structure_literal); - } - /// s3(source, format, structure) - else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) - { - if (second_arg == "auto") - args[1] = format_literal; - if (checkAndGetLiteralArgument(args[2], "structure") == "auto") - args[2] = structure_literal; - } - /// s3(source, access_key_id, access_key_id) -> s3(source, access_key_id, access_key_id, format, structure) - else - { - args.push_back(format_literal); - args.push_back(structure_literal); - } - } - /// s3(source, format, structure, compression_method) or - /// s3(source, access_key_id, secret_access_key, format) or - /// s3(source, NOSIGN, format, structure) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. - else if (count == 4) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - /// s3(source, NOSIGN, format, structure) - if (boost::iequals(second_arg, "NOSIGN")) - { - if (checkAndGetLiteralArgument(args[2], "format") == "auto") - args[2] = format_literal; - if (checkAndGetLiteralArgument(args[3], "structure") == "auto") - args[3] = structure_literal; - } - /// s3(source, format, structure, compression_method) - else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) - { - if (second_arg == "auto") - args[1] = format_literal; - if (checkAndGetLiteralArgument(args[2], "structure") == "auto") - args[2] = structure_literal; - } - /// s3(source, access_key_id, access_key_id, format) -> s3(source, access_key_id, access_key_id, format, structure) - else - { - if (checkAndGetLiteralArgument(args[3], "format") == "auto") - args[3] = format_literal; - args.push_back(structure_literal); - } - } - /// s3(source, access_key_id, secret_access_key, format, structure) or - /// s3(source, NOSIGN, format, structure, compression_method) - /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not. - else if (count == 5) - { - auto sedond_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - /// s3(source, NOSIGN, format, structure, compression_method) - if (boost::iequals(sedond_arg, "NOSIGN")) - { - if (checkAndGetLiteralArgument(args[2], "format") == "auto") - args[2] = format_literal; - if (checkAndGetLiteralArgument(args[3], "structure") == "auto") - args[3] = structure_literal; - } - /// s3(source, access_key_id, access_key_id, format, structure) - else - { - if (checkAndGetLiteralArgument(args[3], "format") == "auto") - args[3] = format_literal; - if (checkAndGetLiteralArgument(args[4], "structure") == "auto") - args[4] = structure_literal; - } - } - /// s3(source, access_key_id, secret_access_key, format, structure, compression) - else if (count == 6) - { - if (checkAndGetLiteralArgument(args[3], "format") == "auto") - args[3] = format_literal; - if (checkAndGetLiteralArgument(args[4], "structure") == "auto") - args[4] = structure_literal; - } - } -} - -ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const -{ - if (configuration.structure == "auto") - { - context->checkAccess(getSourceAccessType()); - configuration.update(context); - if (configuration.format == "auto") - return StorageS3::getTableStructureAndFormatFromData(configuration, std::nullopt, context).first; - - return StorageS3::getTableStructureFromData(configuration, std::nullopt, context); - } - - return parseColumnsListFromString(configuration.structure, context); -} - -bool TableFunctionS3::supportsReadingSubsetOfColumns(const ContextPtr & context) -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context); -} - -std::unordered_set TableFunctionS3::getVirtualsToCheckBeforeUsingStructureHint() const -{ - return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); -} - -StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool /*is_insert_query*/) const -{ - S3::URI s3_uri (configuration.url); - - ColumnsDescription columns; - if (configuration.structure != "auto") - columns = parseColumnsListFromString(configuration.structure, context); - else if (!structure_hint.empty()) - columns = structure_hint; - else if (!cached_columns.empty()) - columns = cached_columns; - - StoragePtr storage = std::make_shared( - configuration, - context, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - String{}, - /// No format_settings for table function S3 - std::nullopt); - - storage->startup(); - - return storage; -} - - -class TableFunctionGCS : public TableFunctionS3 -{ -public: - static constexpr auto name = "gcs"; - std::string getName() const override - { - return name; - } -private: - const char * getStorageTypeName() const override { return "GCS"; } -}; - -class TableFunctionCOS : public TableFunctionS3 -{ -public: - static constexpr auto name = "cosn"; - std::string getName() const override - { - return name; - } -private: - const char * getStorageTypeName() const override { return "COSN"; } -}; - -class TableFunctionOSS : public TableFunctionS3 -{ -public: - static constexpr auto name = "oss"; - std::string getName() const override - { - return name; - } -private: - const char * getStorageTypeName() const override { return "OSS"; } -}; - - -void registerTableFunctionGCS(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description=R"(The table function can be used to read the data stored on Google Cloud Storage.)", - .examples{{"gcs", "SELECT * FROM gcs(url, hmac_key, hmac_secret)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} - -void registerTableFunctionS3(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description=R"(The table function can be used to read the data stored on AWS S3.)", - .examples{{"s3", "SELECT * FROM s3(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} - - -void registerTableFunctionCOS(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -void registerTableFunctionOSS(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h deleted file mode 100644 index 00ca36c6653..00000000000 --- a/src/TableFunctions/TableFunctionS3.h +++ /dev/null @@ -1,86 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include - - -namespace DB -{ - -class Context; - -/* s3(source, [access_key_id, secret_access_key,] [format, structure, compression]) - creates a temporary storage for a file in S3. - */ -class TableFunctionS3 : public ITableFunction -{ -public: - static constexpr auto name = "s3"; - static constexpr auto signature = " - url\n" - " - url, format\n" - " - url, format, structure\n" - " - url, format, structure, compression_method\n" - " - url, access_key_id, secret_access_key\n" - " - url, access_key_id, secret_access_key, session_token\n" - " - url, access_key_id, secret_access_key, format\n" - " - url, access_key_id, secret_access_key, session_token, format\n" - " - url, access_key_id, secret_access_key, format, structure\n" - " - url, access_key_id, secret_access_key, session_token, format, structure\n" - " - url, access_key_id, secret_access_key, format, structure, compression_method\n" - " - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" - "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; - - static size_t getMaxNumberOfArguments() { return 6; } - - String getName() const override - { - return name; - } - - virtual String getSignature() const - { - return signature; - } - - bool hasStaticStructure() const override { return configuration.structure != "auto"; } - - bool needStructureHint() const override { return configuration.structure == "auto"; } - - void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } - - bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; - - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; - - virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); - - static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context); - -protected: - - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "S3"; } - - ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - - mutable StorageS3::Configuration configuration; - ColumnsDescription structure_hint; - -private: - - std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp deleted file mode 100644 index e727c4e4c89..00000000000 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ /dev/null @@ -1,72 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include - -#include "registerTableFunctions.h" - -#include - - -namespace DB -{ - -StoragePtr TableFunctionS3Cluster::executeImpl( - const ASTPtr & /*function*/, ContextPtr context, - const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const -{ - StoragePtr storage; - ColumnsDescription columns; - - if (configuration.structure != "auto") - { - columns = parseColumnsListFromString(configuration.structure, context); - } - else if (!structure_hint.empty()) - { - columns = structure_hint; - } - - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) - { - /// On worker node this filename won't contains globs - storage = std::make_shared( - configuration, - context, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - /* comment */String{}, - /* format_settings */std::nullopt, /// No format_settings for S3Cluster - /*distributed_processing=*/true); - } - else - { - storage = std::make_shared( - cluster_name, - configuration, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - context); - } - - storage->startup(); - - return storage; -} - - -void registerTableFunctionS3Cluster(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - - -} - -#endif diff --git a/src/TableFunctions/TableFunctionS3Cluster.h b/src/TableFunctions/TableFunctionS3Cluster.h deleted file mode 100644 index 718b0d90de8..00000000000 --- a/src/TableFunctions/TableFunctionS3Cluster.h +++ /dev/null @@ -1,64 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include - - -namespace DB -{ - -class Context; - -/** - * s3cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure, compression_method) - * A table function, which allows to process many files from S3 on a specific cluster - * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks - * in S3 file path and dispatch each file dynamically. - * On worker node it asks initiator about next task to process, processes it. - * This is repeated until the tasks are finished. - */ -class TableFunctionS3Cluster : public ITableFunctionCluster -{ -public: - static constexpr auto name = "s3Cluster"; - static constexpr auto signature = " - cluster, url\n" - " - cluster, url, format\n" - " - cluster, url, format, structure\n" - " - cluster, url, access_key_id, secret_access_key\n" - " - cluster, url, format, structure, compression_method\n" - " - cluster, url, access_key_id, secret_access_key, format\n" - " - cluster, url, access_key_id, secret_access_key, format, structure\n" - " - cluster, url, access_key_id, secret_access_key, format, structure, compression_method\n" - " - cluster, url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" - "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; - - String getName() const override - { - return name; - } - - String getSignature() const override - { - return signature; - } - -protected: - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "S3Cluster"; } -}; - -} - -#endif diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp new file mode 100644 index 00000000000..15a6668f434 --- /dev/null +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -0,0 +1,69 @@ +#include +#include + +namespace DB +{ + +#if USE_AWS_S3 +#if USE_AVRO +void registerTableFunctionIceberg(TableFunctionFactory & factory) +{ + factory.registerFunction( + { + .documentation = + { + .description=R"(The table function can be used to read the Iceberg table stored on object store.)", + .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"} + }, + .allow_readonly = false + }); +} +#endif + +#if USE_PARQUET +void registerTableFunctionDeltaLake(TableFunctionFactory & factory) +{ + factory.registerFunction( + { + .documentation = + { + .description=R"(The table function can be used to read the DeltaLake table stored on object store.)", + .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"} + }, + .allow_readonly = false + }); +} +#endif + +void registerTableFunctionHudi(TableFunctionFactory & factory) +{ + factory.registerFunction( + { + .documentation = + { + .description=R"(The table function can be used to read the Hudi table stored on object store.)", + .examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"} + }, + .allow_readonly = false + }); +} +#endif + +void registerDataLakeTableFunctions(TableFunctionFactory & factory) +{ + UNUSED(factory); +#if USE_AWS_S3 +#if USE_AVRO + registerTableFunctionIceberg(factory); +#endif +#if USE_PARQUET + registerTableFunctionDeltaLake(factory); +#endif + registerTableFunctionHudi(factory); +#endif +} + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 927457ff9f6..26b9a771416 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -29,27 +29,6 @@ void registerTableFunctions() registerTableFunctionFuzzJSON(factory); #endif -#if USE_AWS_S3 - registerTableFunctionS3(factory); - registerTableFunctionS3Cluster(factory); - registerTableFunctionCOS(factory); - registerTableFunctionOSS(factory); - registerTableFunctionGCS(factory); - registerTableFunctionHudi(factory); -#if USE_PARQUET - registerTableFunctionDeltaLake(factory); -#endif -#if USE_AVRO - registerTableFunctionIceberg(factory); -#endif - -#endif - -#if USE_HDFS - registerTableFunctionHDFS(factory); - registerTableFunctionHDFSCluster(factory); -#endif - #if USE_HIVE registerTableFunctionHive(factory); #endif @@ -77,12 +56,9 @@ void registerTableFunctions() registerTableFunctionFormat(factory); registerTableFunctionExplain(factory); -#if USE_AZURE_BLOB_STORAGE - registerTableFunctionAzureBlobStorage(factory); - registerTableFunctionAzureBlobStorageCluster(factory); -#endif - - + registerTableFunctionObjectStorage(factory); + registerTableFunctionObjectStorageCluster(factory); + registerDataLakeTableFunctions(factory); } } diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 296af146faf..4a89b3afbb3 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -32,18 +32,6 @@ void registerTableFunctionS3Cluster(TableFunctionFactory & factory); void registerTableFunctionCOS(TableFunctionFactory & factory); void registerTableFunctionOSS(TableFunctionFactory & factory); void registerTableFunctionGCS(TableFunctionFactory & factory); -void registerTableFunctionHudi(TableFunctionFactory & factory); -#if USE_PARQUET -void registerTableFunctionDeltaLake(TableFunctionFactory & factory); -#endif -#if USE_AVRO -void registerTableFunctionIceberg(TableFunctionFactory & factory); -#endif -#endif - -#if USE_HDFS -void registerTableFunctionHDFS(TableFunctionFactory & factory); -void registerTableFunctionHDFSCluster(TableFunctionFactory & factory); #endif #if USE_HIVE @@ -74,10 +62,9 @@ void registerTableFunctionFormat(TableFunctionFactory & factory); void registerTableFunctionExplain(TableFunctionFactory & factory); -#if USE_AZURE_BLOB_STORAGE -void registerTableFunctionAzureBlobStorage(TableFunctionFactory & factory); -void registerTableFunctionAzureBlobStorageCluster(TableFunctionFactory & factory); -#endif +void registerTableFunctionObjectStorage(TableFunctionFactory & factory); +void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory); +void registerDataLakeTableFunctions(TableFunctionFactory & factory); void registerTableFunctions(); diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 4afd3f46f9d..c4e06ccd79a 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -17,7 +17,7 @@ from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union import docker_images_helper import upload_result_helper from build_check import get_release_or_pr -from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames +from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames, StatusNames from ci_utils import GHActions, is_hex, normalize_string from clickhouse_helper import ( CiLogsCredentials, @@ -52,7 +52,7 @@ from git_helper import GIT_PREFIX, Git from git_helper import Runner as GitRunner from github_helper import GitHub from pr_info import PRInfo -from report import ERROR, SUCCESS, BuildResult, JobReport +from report import ERROR, SUCCESS, BuildResult, JobReport, PENDING from s3_helper import S3Helper from ci_metadata import CiMetadata from version_helper import get_version_from_repo @@ -996,6 +996,11 @@ def parse_args(parser: argparse.ArgumentParser) -> argparse.Namespace: action="store_true", help="Action that cancels previous running PR workflow if PR added into the Merge Queue", ) + parser.add_argument( + "--set-pending-status", + action="store_true", + help="Action to set needed pending statuses in the beginning of CI workflow, e.g. for Sync wf", + ) parser.add_argument( "--configure", action="store_true", @@ -1930,6 +1935,20 @@ def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> No ) +def _set_pending_statuses(pr_info: PRInfo) -> None: + commit = get_commit(GitHub(get_best_robot_token(), per_page=100), pr_info.sha) + try: + print("Set SYNC status to pending") + commit.create_status( + state=PENDING, + target_url="", + description="", + context=StatusNames.SYNC, + ) + except Exception as ex: + print(f"ERROR: failed to set GH commit status, ex: {ex}") + + def main() -> int: logging.basicConfig(level=logging.INFO) exit_code = 0 @@ -2265,6 +2284,13 @@ def main() -> int: else: assert False, "BUG! Not supported scenario" + ### SET PENDING STATUS + elif args.set_pending_status: + if pr_info.is_pr: + _set_pending_statuses(pr_info) + else: + assert False, "BUG! Not supported scenario" + ### print results _print_results(result, args.outfile, args.pretty) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index e1c47353743..b17c189c405 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -20,7 +20,6 @@ from github.Repository import Repository from ci_config import CHECK_DESCRIPTIONS, CheckDescription, StatusNames, is_required from env_helper import ( GITHUB_REPOSITORY, - GITHUB_RUN_URL, GITHUB_UPSTREAM_REPOSITORY, TEMP_PATH, ) @@ -433,11 +432,8 @@ def set_mergeable_check( commit: Commit, description: str = "", state: StatusType = SUCCESS, - hide_url: bool = False, ) -> CommitStatus: - report_url = GITHUB_RUN_URL - if hide_url: - report_url = "" + report_url = "" return post_commit_status( commit, state, @@ -469,7 +465,6 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> def trigger_mergeable_check( commit: Commit, statuses: CommitStatuses, - hide_url: bool = False, set_if_green: bool = False, workflow_failed: bool = False, ) -> StatusType: @@ -484,25 +479,30 @@ def trigger_mergeable_check( success = [] fail = [] + pending = [] for status in required_checks: if status.state == SUCCESS: success.append(status.context) + elif status.state == PENDING: + pending.append(status.context) else: fail.append(status.context) state: StatusType = SUCCESS - if success: - description = ", ".join(success) - else: - description = "awaiting job statuses" - if fail: description = "failed: " + ", ".join(fail) state = FAILURE elif workflow_failed: description = "check workflow failures" state = FAILURE + elif pending: + description = "pending: " + ", ".join(pending) + state = PENDING + else: + # all good + description = ", ".join(success) + description = format_description(description) if not set_if_green and state == SUCCESS: @@ -510,7 +510,7 @@ def trigger_mergeable_check( pass else: if mergeable_status is None or mergeable_status.description != description: - set_mergeable_check(commit, description, state, hide_url) + set_mergeable_check(commit, description, state) return state @@ -556,13 +556,12 @@ def update_upstream_sync_status( post_commit_status( last_synced_upstream_commit, sync_status, - "", # let's won't expose any urls from cloud + "", "", StatusNames.SYNC, ) trigger_mergeable_check( last_synced_upstream_commit, get_commit_filtered_statuses(last_synced_upstream_commit), - True, set_if_green=can_set_green_mergeable_status, ) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 1a7000f5353..269d5aa3175 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -15,7 +15,7 @@ from commit_status_helper import ( ) from get_robot_token import get_best_robot_token from pr_info import PRInfo -from report import PENDING, SUCCESS +from report import PENDING from synchronizer_utils import SYNC_BRANCH_PREFIX from env_helper import GITHUB_REPOSITORY, GITHUB_UPSTREAM_REPOSITORY @@ -67,7 +67,7 @@ def main(): if status.state == PENDING: post_commit_status( commit, - SUCCESS, + state, # map Mergeable Check status to CI Running status.target_url, "All checks finished", StatusNames.CI, diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 500de4eb718..e1c7bf94ff5 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -250,7 +250,6 @@ def main(): trigger_mergeable_check( commit, statuses, - hide_url=False, set_if_green=True, workflow_failed=(args.wf_status != "success"), ) diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 206f960293f..7d0127bc1c4 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -165,11 +165,35 @@ class _ServerRuntime: '' "" "ExpectedError" - "mock s3 injected error" + "mock s3 injected unretryable error" "txfbd566d03042474888193-00608d7537" "" ) - request_handler.write_error(data) + request_handler.write_error(500, data) + + class SlowDownAction: + def inject_error(self, request_handler): + data = ( + '' + "" + "SlowDown" + "Slow Down." + "txfbd566d03042474888193-00608d7537" + "" + ) + request_handler.write_error(429, data) + + class QpsLimitExceededAction: + def inject_error(self, request_handler): + data = ( + '' + "" + "QpsLimitExceeded" + "Please reduce your request rate." + "txfbd566d03042474888193-00608d7537" + "" + ) + request_handler.write_error(429, data) class RedirectAction: def __init__(self, host="localhost", port=1): @@ -239,6 +263,12 @@ class _ServerRuntime: self.error_handler = _ServerRuntime.BrokenPipeAction() elif self.action == "redirect_to": self.error_handler = _ServerRuntime.RedirectAction(*self.action_args) + elif self.action == "slow_down": + self.error_handler = _ServerRuntime.SlowDownAction(*self.action_args) + elif self.action == "qps_limit_exceeded": + self.error_handler = _ServerRuntime.QpsLimitExceededAction( + *self.action_args + ) else: self.error_handler = _ServerRuntime.Expected500ErrorAction() @@ -344,12 +374,12 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.end_headers() self.wfile.write(b"Redirected") - def write_error(self, data, content_length=None): + def write_error(self, http_code, data, content_length=None): if content_length is None: content_length = len(data) self.log_message("write_error %s", data) self.read_all_input() - self.send_response(500) + self.send_response(http_code) self.send_header("Content-Type", "text/xml") self.send_header("Content-Length", str(content_length)) self.end_headers() @@ -418,7 +448,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): path = [x for x in parts.path.split("/") if x] assert path[0] == "mock_settings", path if len(path) < 2: - return self.write_error("_mock_settings: wrong command") + return self.write_error(400, "_mock_settings: wrong command") if path[1] == "at_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) @@ -477,7 +507,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.log_message("reset") return self._ok() - return self.write_error("_mock_settings: wrong command") + return self.write_error(400, "_mock_settings: wrong command") def do_GET(self): if self.path == "/": diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml index 4210c13b727..7df7b56b3b4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml @@ -5,6 +5,7 @@ 1000000 1 + 0 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml index 95a313ea4f2..c1ca258f6c4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml @@ -5,6 +5,7 @@ 5 0 + 0 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 22d6d263d23..a7fe02b16de 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -91,7 +91,7 @@ def get_multipart_counters(node, query_id, log_type="ExceptionWhileProcessing"): SELECT ProfileEvents['S3CreateMultipartUpload'], ProfileEvents['S3UploadPart'], - ProfileEvents['S3WriteRequestsErrors'], + ProfileEvents['S3WriteRequestsErrors'] + ProfileEvents['S3WriteRequestsThrottling'], FROM system.query_log WHERE query_id='{query_id}' AND type='{log_type}' @@ -148,7 +148,7 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression ) assert "Code: 499" in error, error - assert "mock s3 injected error" in error, error + assert "mock s3 injected unretryable error" in error, error create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id @@ -190,7 +190,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( ) assert "Code: 499" in error, error - assert "mock s3 injected error" in error, error + assert "mock s3 injected unretryable error" in error, error create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id @@ -200,18 +200,32 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert s3_errors >= 2 -def test_when_s3_connection_refused_is_retried(cluster, broken_s3): +@pytest.mark.parametrize( + "action_and_message", + [ + ("slow_down", "DB::Exception: Slow Down."), + ("qps_limit_exceeded", "DB::Exception: Please reduce your request rate."), + ( + "connection_refused", + "Poco::Exception. Code: 1000, e.code() = 111, Connection refused", + ), + ], + ids=lambda x: x[0], +) +def test_when_error_is_retried(cluster, broken_s3, action_and_message): node = cluster.instances["node"] - broken_s3.setup_fake_multpartuploads() - broken_s3.setup_at_part_upload(count=3, after=2, action="connection_refused") + action, message = action_and_message - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED" + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload(count=3, after=2, action=action) + + insert_query_id = f"INSERT_INTO_TABLE_{action}_RETRIED" node.query( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_refused_at_write_retried', + 'http://resolver:8083/root/data/test_when_{action}_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -234,13 +248,13 @@ def test_when_s3_connection_refused_is_retried(cluster, broken_s3): assert upload_parts == 39 assert s3_errors == 3 - broken_s3.setup_at_part_upload(count=1000, after=2, action="connection_refused") - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED_1" + broken_s3.setup_at_part_upload(count=1000, after=2, action=action) + insert_query_id = f"INSERT_INTO_TABLE_{action}_RETRIED_1" error = node.query_and_get_error( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_refused_at_write_retried', + 'http://resolver:8083/root/data/test_when_{action}_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -257,8 +271,78 @@ def test_when_s3_connection_refused_is_retried(cluster, broken_s3): ) assert "Code: 499" in error, error + assert message in error, error + + +def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload( + count=3, + after=2, + action="broken_pipe", + ) + + insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=1000000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + create_multipart, upload_parts, s3_errors = get_multipart_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert create_multipart == 1 + assert upload_parts == 7 + assert s3_errors == 3 + + broken_s3.setup_at_part_upload( + count=1000, + after=2, + action="broken_pipe", + ) + insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=1000000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 1000" in error, error assert ( - "Poco::Exception. Code: 1000, e.code() = 111, Connection refused" in error + "DB::Exception: Poco::Exception. Code: 1000, e.code() = 32, I/O error: Broken pipe" + in error ), error @@ -401,20 +485,20 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( ) error = node.query_and_get_error( f""" - INSERT INTO - TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_create_mpu_retried', - 'minio', 'minio123', - 'CSV', auto, 'none' - ) - SELECT - * - FROM system.numbers - LIMIT 1000 - SETTINGS - s3_max_single_part_upload_size=100, - s3_min_upload_part_size=100, - s3_check_objects_after_upload=0 + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_create_mpu_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 """, query_id=insert_query_id, ) @@ -427,78 +511,6 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( ), error -def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): - node = cluster.instances["node"] - - broken_s3.setup_fake_multpartuploads() - broken_s3.setup_at_part_upload( - count=3, - after=2, - action="broken_pipe", - ) - - insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD" - node.query( - f""" - INSERT INTO - TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', - 'minio', 'minio123', - 'CSV', auto, 'none' - ) - SELECT - * - FROM system.numbers - LIMIT 1000000 - SETTINGS - s3_max_single_part_upload_size=100, - s3_min_upload_part_size=1000000, - s3_check_objects_after_upload=0 - """, - query_id=insert_query_id, - ) - - create_multipart, upload_parts, s3_errors = get_multipart_counters( - node, insert_query_id, log_type="QueryFinish" - ) - - assert create_multipart == 1 - assert upload_parts == 7 - assert s3_errors == 3 - - broken_s3.setup_at_part_upload( - count=1000, - after=2, - action="broken_pipe", - ) - insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1" - error = node.query_and_get_error( - f""" - INSERT INTO - TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', - 'minio', 'minio123', - 'CSV', auto, 'none' - ) - SELECT - * - FROM system.numbers - LIMIT 1000000 - SETTINGS - s3_max_single_part_upload_size=100, - s3_min_upload_part_size=1000000, - s3_check_objects_after_upload=0 - """, - query_id=insert_query_id, - ) - - assert "Code: 1000" in error, error - assert ( - "DB::Exception: Poco::Exception. Code: 1000, e.code() = 32, I/O error: Broken pipe" - in error - ), error - - def test_query_is_canceled_with_inf_retries(cluster, broken_s3): node = cluster.instances["node_with_inf_s3_retries"] diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 9216b08f942..0bf81e81383 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -857,9 +857,9 @@ def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_poli error = node.query_and_get_error( "OPTIMIZE TABLE test_merge_canceled_by_s3_errors FINAL", ) - assert "ExpectedError Message: mock s3 injected error" in error, error + assert "ExpectedError Message: mock s3 injected unretryable error" in error, error - node.wait_for_log_line("ExpectedError Message: mock s3 injected error") + node.wait_for_log_line("ExpectedError Message: mock s3 injected unretryable error") table_uuid = node.query( "SELECT uuid FROM system.tables WHERE database = 'default' AND name = 'test_merge_canceled_by_s3_errors' LIMIT 1" @@ -867,7 +867,7 @@ def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_poli node.query("SYSTEM FLUSH LOGS") error_count_in_blob_log = node.query( - f"SELECT count() FROM system.blob_storage_log WHERE query_id like '{table_uuid}::%' AND error like '%mock s3 injected error%'" + f"SELECT count() FROM system.blob_storage_log WHERE query_id like '{table_uuid}::%' AND error like '%mock s3 injected unretryable error%'" ).strip() assert int(error_count_in_blob_log) > 0, node.query( f"SELECT * FROM system.blob_storage_log WHERE query_id like '{table_uuid}::%' FORMAT PrettyCompactMonoBlock" @@ -911,7 +911,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): node.query("OPTIMIZE TABLE merge_canceled_by_s3_errors_when_move FINAL") - node.wait_for_log_line("ExpectedError Message: mock s3 injected error") + node.wait_for_log_line("ExpectedError Message: mock s3 injected unretryable error") count = node.query("SELECT count() FROM merge_canceled_by_s3_errors_when_move") assert int(count) == 2000, count diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 78aaf26a2a7..f836c58ce30 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -30,6 +30,8 @@ def cluster(): with_azurite=True, ) cluster.start() + container_client = cluster.blob_service_client.get_container_client("cont") + container_client.create_container() yield cluster finally: cluster.shutdown() @@ -130,8 +132,10 @@ def test_create_table_connection_string(cluster): node = cluster.instances["node"] azure_query( node, - f"CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}'," - f"'cont', 'test_create_connection_string', 'CSV')", + f""" + CREATE TABLE test_create_table_conn_string (key UInt64, data String) + Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_create_connection_string', 'CSV') + """, ) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 3c43918d8c0..44c0223e677 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -61,7 +61,12 @@ def test_read_write_storage_with_globs(started_cluster): hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" - assert node1.query("select count(*) from HDFSStorageWithRange") == "3\n" + assert ( + node1.query( + "select count(*) from HDFSStorageWithRange settings s3_throw_on_zero_files_match=1" + ) + == "3\n" + ) assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n" assert node1.query("select count(*) from HDFSStorageWithQuestionMark") == "3\n" assert node1.query("select count(*) from HDFSStorageWithAsterisk") == "3\n" @@ -159,7 +164,7 @@ def test_bad_hdfs_uri(started_cluster): ) except Exception as ex: print(ex) - assert "Unable to create builder to connect to HDFS" in str(ex) + assert "Unable to connect to HDFS" in str(ex) try: node1.query( @@ -321,7 +326,7 @@ def test_virtual_columns(started_cluster): hdfs_api.write_data("/file1", "1\n") hdfs_api.write_data("/file2", "2\n") hdfs_api.write_data("/file3", "3\n") - expected = "1\tfile1\thdfs://hdfs1:9000/file1\n2\tfile2\thdfs://hdfs1:9000/file2\n3\tfile3\thdfs://hdfs1:9000/file3\n" + expected = "1\tfile1\tfile1\n2\tfile2\tfile2\n3\tfile3\tfile3\n" assert ( node1.query( "select id, _file as file_name, _path as file_path from virtual_cols order by id" @@ -360,7 +365,12 @@ def test_truncate_table(started_cluster): assert hdfs_api.read_data("/tr") == "1\tMark\t72.53\n" assert node1.query("select * from test_truncate") == "1\tMark\t72.53\n" node1.query("truncate table test_truncate") - assert node1.query("select * from test_truncate") == "" + assert ( + node1.query( + "select * from test_truncate settings hdfs_ignore_file_doesnt_exist=1" + ) + == "" + ) node1.query("drop table test_truncate") @@ -483,13 +493,13 @@ def test_hdfsCluster(started_cluster): actual = node1.query( "select id, _file as file_name, _path as file_path from hdfs('hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') order by id" ) - expected = "1\tfile1\thdfs://hdfs1:9000/test_hdfsCluster/file1\n2\tfile2\thdfs://hdfs1:9000/test_hdfsCluster/file2\n3\tfile3\thdfs://hdfs1:9000/test_hdfsCluster/file3\n" + expected = "1\tfile1\ttest_hdfsCluster/file1\n2\tfile2\ttest_hdfsCluster/file2\n3\tfile3\ttest_hdfsCluster/file3\n" assert actual == expected actual = node1.query( "select id, _file as file_name, _path as file_path from hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') order by id" ) - expected = "1\tfile1\thdfs://hdfs1:9000/test_hdfsCluster/file1\n2\tfile2\thdfs://hdfs1:9000/test_hdfsCluster/file2\n3\tfile3\thdfs://hdfs1:9000/test_hdfsCluster/file3\n" + expected = "1\tfile1\ttest_hdfsCluster/file1\n2\tfile2\ttest_hdfsCluster/file2\n3\tfile3\ttest_hdfsCluster/file3\n" assert actual == expected fs.delete(dir, recursive=True) @@ -497,7 +507,9 @@ def test_hdfsCluster(started_cluster): def test_hdfs_directory_not_exist(started_cluster): ddl = "create table HDFSStorageWithNotExistDir (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/data/not_eixst', 'TSV')" node1.query(ddl) - assert "" == node1.query("select * from HDFSStorageWithNotExistDir") + assert "" == node1.query( + "select * from HDFSStorageWithNotExistDir settings hdfs_ignore_file_doesnt_exist=1" + ) def test_overwrite(started_cluster): @@ -653,7 +665,7 @@ def test_virtual_columns_2(started_cluster): node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") result = node1.query(f"SELECT _path FROM {table_function}") - assert result.strip() == "hdfs://hdfs1:9000/parquet_2" + assert result.strip() == "parquet_2" table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" @@ -966,37 +978,25 @@ def test_read_subcolumns(started_cluster): f"select a.b.d, _path, a.b, _file, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert ( - res - == "2\thdfs://hdfs1:9000/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" - ) + assert res == "2\ttest_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" res = node.query( f"select a.b.d, _path, a.b, _file, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert ( - res - == "2\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" - ) + assert res == "2\ttest_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert ( - res - == "0\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" - ) + assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) - assert ( - res - == "42\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" - ) + assert res == "42\ttest_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" def test_union_schema_inference_mode(started_cluster): diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py index c72152fa376..ddfc1f6483d 100644 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ b/tests/integration/test_storage_kerberized_hdfs/test.py @@ -130,7 +130,7 @@ def test_prohibited(started_cluster): assert False, "Exception have to be thrown" except Exception as ex: assert ( - "Unable to open HDFS file: /storage_user_two_prohibited error: Permission denied: user=specuser, access=WRITE" + "Unable to open HDFS file: /storage_user_two_prohibited (hdfs://suser@kerberizedhdfs1:9010/storage_user_two_prohibited) error: Permission denied: user=specuser, access=WRITE" in str(ex) ) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index dc929b7db46..09b27fff1e8 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1816,27 +1816,13 @@ def test_schema_inference_cache(started_cluster): check_cache(instance, []) run_describe_query(instance, files, storage_name, started_cluster, bucket) - check_cache_misses( - instance, - files, - storage_name, - started_cluster, - bucket, - 4 if storage_name == "url" else 1, - ) + check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4) instance.query("system drop schema cache") check_cache(instance, []) run_describe_query(instance, files, storage_name, started_cluster, bucket) - check_cache_misses( - instance, - files, - storage_name, - started_cluster, - bucket, - 4 if storage_name == "url" else 1, - ) + check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4) instance.query("system drop schema cache") diff --git a/tests/queries/0_stateless/00910_buffer_prewhere_different_types.sql b/tests/queries/0_stateless/00910_buffer_prewhere_different_types.sql index 8f305914cb8..702d9bb3e6c 100644 --- a/tests/queries/0_stateless/00910_buffer_prewhere_different_types.sql +++ b/tests/queries/0_stateless/00910_buffer_prewhere_different_types.sql @@ -2,8 +2,14 @@ DROP TABLE IF EXISTS buffer_table1__fuzz_28; DROP TABLE IF EXISTS merge_tree_table1; CREATE TABLE merge_tree_table1 (`x` UInt32) ENGINE = MergeTree ORDER BY x; + +CREATE TABLE buffer_table1__fuzz_24 (`s` Nullable(Int128), `x` Nullable(FixedString(17))) ENGINE = Buffer(currentDatabase(), 'merge_tree_table1', 16, 10, 60, 10, 1000, 1048576, 2097152); +SELECT s FROM buffer_table1__fuzz_24 PREWHERE factorial(toNullable(10)); -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } + INSERT INTO merge_tree_table1 VALUES (1), (2), (3), (4), (5), (6), (7), (8), (9), (10); +SELECT s FROM buffer_table1__fuzz_24 PREWHERE factorial(toNullable(10)); -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } + SET send_logs_level='error'; CREATE TABLE buffer_table1__fuzz_28 (`x` Nullable(UInt32)) ENGINE = Buffer(currentDatabase(), 'merge_tree_table1', 16, 10, 60, 10, 1000, 1048576, 2097152); diff --git a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.reference b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.reference new file mode 100644 index 00000000000..f572a3570f4 --- /dev/null +++ b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.reference @@ -0,0 +1,19 @@ +------------------- Distributed ------------------ +1 +---------- merge() over distributed -------------- +2 +---------- merge() over local -------------------- +1 +1 +1 +---------- remote() over Merge ------------------- +2 +---------- Distributed over Merge ---------------- +1 +---------- remote() over Merge ------------------- +2 +---------- Merge over Distributed ----------------- +1 +1 +1 +2 diff --git a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql new file mode 100644 index 00000000000..6b0dd4c8747 --- /dev/null +++ b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql @@ -0,0 +1,83 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/64211 +DROP TABLE IF EXISTS test_merge; +DROP TABLE IF EXISTS test_merge_distributed; +DROP TABLE IF EXISTS test_distributed_merge; +DROP TABLE IF EXISTS test_distributed; +DROP TABLE IF EXISTS test_local; +CREATE TABLE test_local (name String) +ENGINE = MergeTree +ORDER BY name as select 'x'; + +CREATE TABLE test_distributed as test_local +ENGINE = Distributed(test_shard_localhost, currentDatabase(), test_local); + +CREATE TABLE test_merge as test_local +ENGINE = Merge(currentDatabase(), 'test_local'); + +CREATE TABLE test_merge_distributed as test_local +ENGINE = Distributed(test_shard_localhost, currentDatabase(), test_merge); + +CREATE TABLE test_distributed_merge as test_local +ENGINE = Merge(currentDatabase(), 'test_distributed'); + +SELECT '------------------- Distributed ------------------'; +SELECT count() +FROM test_distributed +WHERE name GLOBAL IN (SELECT name FROM test_distributed); + +SELECT '---------- merge() over distributed --------------'; +SELECT count() +FROM merge(currentDatabase(), 'test_distributed') +WHERE name GLOBAL IN (SELECT name FROM test_distributed); + +SELECT '---------- merge() over local --------------------'; +SELECT count() +FROM merge(currentDatabase(), 'test_local') +WHERE name GLOBAL IN (SELECT name FROM test_distributed); + +SELECT count() +FROM merge(currentDatabase(), 'test_local') +WHERE name GLOBAL IN (SELECT name FROM merge(currentDatabase(), 'test_local')); + +SELECT count() +FROM merge(currentDatabase(), 'test_local') +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge)); + +SELECT '---------- remote() over Merge -------------------'; +SELECT count() +FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge) +WHERE name GLOBAL IN (SELECT name FROM test_distributed); + +SELECT '---------- Distributed over Merge ----------------'; +SELECT count() +FROM test_merge_distributed +WHERE name GLOBAL IN (SELECT name FROM test_merge_distributed); + +SELECT '---------- remote() over Merge -------------------'; +SELECT count() +FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge) +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge)); + +SELECT '---------- Merge over Distributed -----------------'; +SELECT count() +FROM test_distributed_merge +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge)); + +SELECT count() +FROM test_distributed_merge +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_distributed_merge)); + +SELECT count() +FROM test_distributed_merge +WHERE name GLOBAL IN (SELECT name FROM test_distributed_merge); + +SELECT count() +FROM remote('127.0.0.{1,2}', currentDatabase(), test_distributed_merge) +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge)); + + +DROP TABLE test_merge; +DROP TABLE test_merge_distributed; +DROP TABLE test_distributed_merge; +DROP TABLE test_distributed; +DROP TABLE test_local; diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index 67a2a70b509..1c1eb4489ee 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -11,7 +11,7 @@ REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") # Check that if we have one inactive replica and a huge number of INSERTs to active replicas, # the number of nodes in ZooKeeper does not grow unbounded. -SCALE=5000 +SCALE=1000 $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS r1; diff --git a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh index acb2550d48c..85fdf7ed764 100755 --- a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh +++ b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh @@ -11,14 +11,24 @@ CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()" -for _ in {1..100}; do - ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" - ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" - ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL" 2>/dev/null & - ${CLICKHOUSE_CLIENT} --query="ALTER TABLE t DETACH PARTITION tuple()" - ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM t HAVING count() > 0" -done +function thread_ops() +{ + local TIMELIMIT=$((SECONDS+$1)) + local it=0 + while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 100 ]; + do + it=$((it+1)) + ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" + ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" + ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL" 2>/dev/null & + ${CLICKHOUSE_CLIENT} --query="ALTER TABLE t DETACH PARTITION tuple()" + ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM t HAVING count() > 0" + done +} +export -f thread_ops +TIMEOUT=60 +thread_ops $TIMEOUT & wait $CLICKHOUSE_CLIENT -q "DROP TABLE t" diff --git a/tests/queries/0_stateless/02114_hdfs_bad_url.sh b/tests/queries/0_stateless/02114_hdfs_bad_url.sh index 22975dddf6f..5bd5610a9f0 100755 --- a/tests/queries/0_stateless/02114_hdfs_bad_url.sh +++ b/tests/queries/0_stateless/02114_hdfs_bad_url.sh @@ -23,4 +23,3 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM hdfs('hdfs1:9000/data', 'CSV', 'x UInt32')" $CLICKHOUSE_CLIENT -q "SELECT * FROM hdfs('hdfs://hdfs1/data', 'CSV', 'x UInt32')" 2>&1 | grep -F -q "HDFS_ERROR" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT -q "SELECT * FROM hdfs('http://hdfs1:9000/data', 'CSV', 'x UInt32')" 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT -q "SELECT * FROM hdfs('hdfs://hdfs1@nameservice/abcd/data', 'CSV', 'x UInt32')" 2>&1 | grep -F -q "HDFS_ERROR" && echo 'OK' || echo 'FAIL'; - diff --git a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql index 8924627a717..26a201ec89f 100644 --- a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql +++ b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql @@ -1,16 +1,16 @@ -- Tags: long, no-parallel -SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 1024 partitions, so disable fault injections +SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 100 partitions, so disable fault injections -- regression for MEMORY_LIMIT_EXCEEDED error because of deferred final part flush drop table if exists data_02228; -create table data_02228 (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by key1 % 1024; -insert into data_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; -insert into data_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } +create table data_02228 (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by key1 % 100; +insert into data_02228 select number, 1, number from numbers_mt(10_000) settings max_memory_usage='30Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; +insert into data_02228 select number, 1, number from numbers_mt(10_000) settings max_memory_usage='30Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=1000000; -- { serverError MEMORY_LIMIT_EXCEEDED } drop table data_02228; drop table if exists data_rep_02228 SYNC; -create table data_rep_02228 (key1 UInt32, sign Int8, s UInt64) engine = ReplicatedCollapsingMergeTree('/clickhouse/{database}', 'r1', sign) order by (key1) partition by key1 % 1024; -insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; -insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } +create table data_rep_02228 (key1 UInt32, sign Int8, s UInt64) engine = ReplicatedCollapsingMergeTree('/clickhouse/{database}', 'r1', sign) order by (key1) partition by key1 % 100; +insert into data_rep_02228 select number, 1, number from numbers_mt(10_000) settings max_memory_usage='30Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; +insert into data_rep_02228 select number, 1, number from numbers_mt(10_000) settings max_memory_usage='30Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=1000000; -- { serverError MEMORY_LIMIT_EXCEEDED } drop table data_rep_02228 SYNC; diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index 050b8e37722..f82f79dbe44 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -102,7 +102,7 @@ ALTER TABLE t_proj ADD PROJECTION p_1 (SELECT avg(a), avg(b), count()) SETTINGS INSERT INTO t_proj SELECT number + 1, number + 1 FROM numbers(1000); -DELETE FROM t_proj WHERE a < 100; -- { serverError BAD_ARGUMENTS } +DELETE FROM t_proj WHERE a < 100; -- { serverError NOT_IMPLEMENTED } SELECT avg(a), avg(b), count() FROM t_proj; diff --git a/tests/queries/0_stateless/02344_insert_profile_events_stress.sql b/tests/queries/0_stateless/02344_insert_profile_events_stress.sql index f9fdd3b943f..e9a790bea5d 100644 --- a/tests/queries/0_stateless/02344_insert_profile_events_stress.sql +++ b/tests/queries/0_stateless/02344_insert_profile_events_stress.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, long, no-debug, no-tsan +-- Tags: no-parallel, long, no-debug, no-tsan, no-msan, no-asan create table data_02344 (key Int) engine=Null; -- 3e9 rows is enough to fill the socket buffer and cause INSERT hung. diff --git a/tests/queries/0_stateless/02494_query_cache_key.reference b/tests/queries/0_stateless/02494_query_cache_key.reference new file mode 100644 index 00000000000..8f5b61192d5 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_key.reference @@ -0,0 +1,6 @@ +Test (1) +1 +2 +Test (2) +4 +4 diff --git a/tests/queries/0_stateless/02494_query_cache_key.sql b/tests/queries/0_stateless/02494_query_cache_key.sql new file mode 100644 index 00000000000..d8c68e0d267 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_key.sql @@ -0,0 +1,70 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +-- Tests that the key of the query cache is not only formed by the query AST but also by +-- (1) the current database (`USE db`, issue #64136), +-- (2) the query settings + + +SELECT 'Test (1)'; + +SYSTEM DROP QUERY CACHE; + +DROP DATABASE IF EXISTS db1; +DROP DATABASE IF EXISTS db2; + +CREATE DATABASE db1; +CREATE DATABASE db2; + +CREATE TABLE db1.tab(a UInt64, PRIMARY KEY a); +CREATE TABLE db2.tab(a UInt64, PRIMARY KEY a); + +INSERT INTO db1.tab values(1); +INSERT INTO db2.tab values(2); + +USE db1; +SELECT * FROM tab SETTINGS use_query_cache=1; + +USE db2; +SELECT * FROM tab SETTINGS use_query_cache=1; + +DROP DATABASE db1; +DROP DATABASE db2; + +SYSTEM DROP QUERY CACHE; + + +SELECT 'Test (2)'; + +-- test with query-level settings +SELECT 1 SETTINGS use_query_cache = 1, limit = 1, use_skip_indexes = 0 Format Null; +SELECT 1 SETTINGS use_query_cache = 1, use_skip_indexes = 0 Format Null; +SELECT 1 SETTINGS use_query_cache = 1, use_skip_indexes = 1 Format Null; +SELECT 1 SETTINGS use_query_cache = 1, max_block_size = 1 Format Null; + +-- 4x the same query but with different settings each. There should yield four entries in the query cache. +SELECT count(query) FROM system.query_cache; + +SYSTEM DROP QUERY CACHE; + +-- test with mixed session-level/query-level settings +SET use_query_cache = 1; +SET limit = 1; +SELECT 1 SETTINGS use_skip_indexes = 0 Format Null; +SET limit = default; +SET use_skip_indexes = 0; +SELECT 1 Format Null; +SET use_skip_indexes = 1; +SELECT 1 SETTINGS use_skip_indexes = 1 Format Null; +SET use_skip_indexes = default; +SET max_block_size = 1; +SELECT 1 Format Null; +SET max_block_size = default; + +SET use_query_cache = default; + +-- 4x the same query but with different settings each. There should yield four entries in the query cache. +SELECT count(query) FROM system.query_cache; + +SYSTEM DROP QUERY CACHE; + diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference index 389e2621455..9ec033cefb1 100644 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference @@ -1,2 +1,4 @@ 2 0 +1 +0 diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index 8712c7c84c6..6bc3d03ac66 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -15,11 +15,17 @@ ${CLICKHOUSE_CLIENT} --query "CREATE TABLE tab (a UInt64) ENGINE=MergeTree() ORD ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (1) (2) (3)" ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" -SETTINGS="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" +SETTINGS_NO_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" +SETTINGS_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l + +${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" + +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" diff --git a/tests/queries/0_stateless/02494_query_cache_use_database.reference b/tests/queries/0_stateless/02494_query_cache_use_database.reference deleted file mode 100644 index 1191247b6d9..00000000000 --- a/tests/queries/0_stateless/02494_query_cache_use_database.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -2 diff --git a/tests/queries/0_stateless/02494_query_cache_use_database.sql b/tests/queries/0_stateless/02494_query_cache_use_database.sql deleted file mode 100644 index df560f82ebb..00000000000 --- a/tests/queries/0_stateless/02494_query_cache_use_database.sql +++ /dev/null @@ -1,30 +0,0 @@ --- Tags: no-parallel, no-fasttest --- Tag no-fasttest: Depends on OpenSSL --- Tag no-parallel: Messes with internal cache - --- Test for issue #64136 - -SYSTEM DROP QUERY CACHE; - -DROP DATABASE IF EXISTS db1; -DROP DATABASE IF EXISTS db2; - -CREATE DATABASE db1; -CREATE DATABASE db2; - -CREATE TABLE db1.tab(a UInt64, PRIMARY KEY a); -CREATE TABLE db2.tab(a UInt64, PRIMARY KEY a); - -INSERT INTO db1.tab values(1); -INSERT INTO db2.tab values(2); - -USE db1; -SELECT * FROM tab SETTINGS use_query_cache=1; - -USE db2; -SELECT * FROM tab SETTINGS use_query_cache=1; - -DROP DATABASE db1; -DROP DATABASE db2; - -SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh b/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh index d831c7d9806..a34a480a078 100755 --- a/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh +++ b/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh @@ -13,7 +13,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -nm -q " INSERT INTO FUNCTION s3('http://localhost:11111/test/$CLICKHOUSE_DATABASE/test_INT_MAX.tsv', '', '', 'TSV') SELECT repeat('a', 1024) FROM numbers((pow(2, 30) * 2) / 1024) - SETTINGS s3_max_single_part_upload_size = '10Gi'; + SETTINGS s3_max_single_part_upload_size = '5Gi'; SELECT count() FROM s3('http://localhost:11111/test/$CLICKHOUSE_DATABASE/test_INT_MAX.tsv'); " diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index d62f928e947..1eb22976b84 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -58,9 +58,8 @@ SELECT * FROM \"abacaba/file.tsv\" """ 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" ${CLICKHOUSE_CLIENT} -q "SELECT * FROM test_hdfs_4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: - +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "The data format cannot be detected" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "The table structure cannot be extracted" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index fe45a2a317d..9320d0e57c3 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -41,7 +41,7 @@ create temporary table basic_types_02735 as select * from generateRandom(' decimal128 Decimal128(20), decimal256 Decimal256(40), ipv4 IPv4, - ipv6 IPv6') limit 10101; + ipv6 IPv6') limit 1011; insert into function file(basic_types_02735.parquet) select * from basic_types_02735; desc file(basic_types_02735.parquet); select (select sum(cityHash64(*)) from basic_types_02735) - (select sum(cityHash64(*)) from file(basic_types_02735.parquet)); @@ -59,7 +59,7 @@ create temporary table nullables_02735 as select * from generateRandom(' fstr Nullable(FixedString(12)), i256 Nullable(Int256), decimal256 Nullable(Decimal256(40)), - ipv6 Nullable(IPv6)') limit 10000; + ipv6 Nullable(IPv6)') limit 1000; insert into function file(nullables_02735.parquet) select * from nullables_02735; select (select sum(cityHash64(*)) from nullables_02735) - (select sum(cityHash64(*)) from file(nullables_02735.parquet)); drop table nullables_02735; @@ -83,7 +83,7 @@ create table arrays_02735 engine = Memory as select * from generateRandom(' decimal64 Array(Decimal64(10)), ipv4 Array(IPv4), msi Map(String, Int16), - tup Tuple(FixedString(3), Array(String), Map(Int8, Date))') limit 10000; + tup Tuple(FixedString(3), Array(String), Map(Int8, Date))') limit 1000; insert into function file(arrays_02735.parquet) select * from arrays_02735; create temporary table arrays_out_02735 as arrays_02735; insert into arrays_out_02735 select * from file(arrays_02735.parquet); @@ -107,7 +107,7 @@ create temporary table madness_02735 as select * from generateRandom(' mln Map(LowCardinality(String), Nullable(Int8)), t Tuple(Map(FixedString(5), Tuple(Array(UInt16), Nullable(UInt16), Array(Tuple(Int8, Decimal64(10))))), Tuple(kitchen UInt64, sink String)), n Nested(hello UInt64, world Tuple(first String, second FixedString(1))) - ') limit 10000; + ') limit 1000; insert into function file(madness_02735.parquet) select * from madness_02735; insert into function file(a.csv) select * from madness_02735 order by tuple(*); insert into function file(b.csv) select aa, aaa, an, aan, l, ln, arrayMap(x->reinterpret(x, 'UInt128'), al) as al_, aaln, mln, t, n.hello, n.world from file(madness_02735.parquet) order by tuple(aa, aaa, an, aan, l, ln, al_, aaln, mln, t, n.hello, n.world); diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.sql b/tests/queries/0_stateless/02792_drop_projection_lwd.sql index a1d8a9c90f3..dcde7dcc600 100644 --- a/tests/queries/0_stateless/02792_drop_projection_lwd.sql +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.sql @@ -7,7 +7,7 @@ CREATE TABLE t_projections_lwd (a UInt32, b UInt32, PROJECTION p (SELECT * ORDER INSERT INTO t_projections_lwd SELECT number, number FROM numbers(100); -- LWD does not work, as expected -DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError BAD_ARGUMENTS } +DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError NOT_IMPLEMENTED } KILL MUTATION WHERE database = currentDatabase() AND table = 't_projections_lwd' SYNC FORMAT Null; -- drop projection diff --git a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql index da76a5cb88f..1e99eb8b83d 100644 --- a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql +++ b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql @@ -2,5 +2,4 @@ select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } -select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } - +select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_DETECT_FORMAT } diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference new file mode 100644 index 00000000000..f5b7872f81e --- /dev/null +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -0,0 +1,9 @@ +-- generateSnowflakeID +1 +0 +0 +1 +100 +-- generateSnowflakeIDThreadMonotonic +1 +100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql new file mode 100644 index 00000000000..57cdd21a9fe --- /dev/null +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -0,0 +1,29 @@ +SELECT '-- generateSnowflakeID'; + +SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero + +SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -- disabled common subexpression elimination --> lhs != rhs +SELECT generateSnowflakeID() = generateSnowflakeID(1); -- same as ^^ +SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- enabled common subexpression elimination + +SELECT generateSnowflakeID(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT count(*) +FROM +( + SELECT DISTINCT generateSnowflakeID() + FROM numbers(100) +); + +SELECT '-- generateSnowflakeIDThreadMonotonic'; + +SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeIDThreadMonotonic()), 63), 1) = 0; -- check first bit is zero + +SELECT generateSnowflakeIDThreadMonotonic(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT count(*) +FROM +( + SELECT DISTINCT generateSnowflakeIDThreadMonotonic() + FROM numbers(100) +); diff --git a/tests/queries/0_stateless/03147_system_columns_access_checks.sh b/tests/queries/0_stateless/03147_system_columns_access_checks.sh index 2bd7fb083ea..b027ea28504 100755 --- a/tests/queries/0_stateless/03147_system_columns_access_checks.sh +++ b/tests/queries/0_stateless/03147_system_columns_access_checks.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-ordinary-database, long +# Tags: no-fasttest, no-parallel, no-ordinary-database, long, no-debug, no-asan, no-tsan, no-msan CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.reference b/tests/queries/0_stateless/03155_analyzer_interpolate.reference new file mode 100644 index 00000000000..791aaa5b2a2 --- /dev/null +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.reference @@ -0,0 +1,13 @@ +0 [5] +0.5 [5] +1 [1] +1.5 [5] +2 [5] +2.5 [5] +3 [5] +3.5 [5] +4 [4] +4.5 [5] +5 [5] +5.5 [5] +7 [7] diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.sql b/tests/queries/0_stateless/03155_analyzer_interpolate.sql new file mode 100644 index 00000000000..b3c1d233f47 --- /dev/null +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.sql @@ -0,0 +1,12 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/62464 +SET allow_experimental_analyzer = 1; + +SELECT n, [number] AS inter FROM ( + SELECT toFloat32(number % 10) AS n, number + FROM numbers(10) WHERE number % 3 = 1 +) GROUP BY n, inter ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS [5]); + +SELECT n, number+5 AS inter FROM ( -- { serverError NOT_AN_AGGREGATE } + SELECT toFloat32(number % 10) AS n, number, number*2 AS mn + FROM numbers(10) WHERE number % 3 = 1 +) GROUP BY n, inter ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS mn * 2); diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql new file mode 100644 index 00000000000..cd29fae8fd7 --- /dev/null +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -0,0 +1,15 @@ + +DROP TABLE IF EXISTS users; + +CREATE TABLE users ( + uid Int16, + name String, + age Int16, + projection p1 (select count(), age group by age) +) ENGINE = MergeTree order by uid; + +INSERT INTO users VALUES (1231, 'John', 33); +INSERT INTO users VALUES (6666, 'Ksenia', 48); +INSERT INTO users VALUES (8888, 'Alice', 50); + +DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 6df2e426561..8f8d74f39ad 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -246,6 +246,7 @@ DockerHub DoubleDelta Doxygen Durre +doesnt ECMA Ecto EdgeAngle @@ -1617,6 +1618,8 @@ gcem generateRandom generateRandomStructure generateSeries +generateSnowflakeID +generateSnowflakeIDThreadMonotonic generateULID generateUUIDv geoDistance diff --git a/utils/keeper-bench/CMakeLists.txt b/utils/keeper-bench/CMakeLists.txt index 5514c34f4ef..4fe0d852fd2 100644 --- a/utils/keeper-bench/CMakeLists.txt +++ b/utils/keeper-bench/CMakeLists.txt @@ -4,5 +4,4 @@ if (NOT TARGET ch_contrib::rapidjson) endif () clickhouse_add_executable(keeper-bench Generator.cpp Runner.cpp Stats.cpp main.cpp) -target_link_libraries(keeper-bench PRIVATE dbms) -target_link_libraries(keeper-bench PRIVATE ch_contrib::rapidjson) +target_link_libraries(keeper-bench PRIVATE dbms clickhouse_functions ch_contrib::rapidjson) diff --git a/utils/keeper-bench/Generator.cpp b/utils/keeper-bench/Generator.cpp index 2212f7158ae..cbf1bcdae23 100644 --- a/utils/keeper-bench/Generator.cpp +++ b/utils/keeper-bench/Generator.cpp @@ -40,54 +40,6 @@ std::string generateRandomString(size_t length) } } -void removeRecursive(Coordination::ZooKeeper & zookeeper, const std::string & path) -{ - namespace fs = std::filesystem; - - auto promise = std::make_shared>(); - auto future = promise->get_future(); - - Strings children; - auto list_callback = [promise, &children] (const ListResponse & response) - { - children = response.names; - - promise->set_value(); - }; - zookeeper.list(path, ListRequestType::ALL, list_callback, nullptr); - future.get(); - - while (!children.empty()) - { - Coordination::Requests ops; - for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) - { - removeRecursive(zookeeper, fs::path(path) / children.back()); - ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); - children.pop_back(); - } - auto multi_promise = std::make_shared>(); - auto multi_future = multi_promise->get_future(); - - auto multi_callback = [multi_promise] (const MultiResponse &) - { - multi_promise->set_value(); - }; - zookeeper.multi(ops, multi_callback); - multi_future.get(); - } - auto remove_promise = std::make_shared>(); - auto remove_future = remove_promise->get_future(); - - auto remove_callback = [remove_promise] (const RemoveResponse &) - { - remove_promise->set_value(); - }; - - zookeeper.remove(path, -1, remove_callback); - remove_future.get(); -} - NumberGetter NumberGetter::fromConfig(const std::string & key, const Poco::Util::AbstractConfiguration & config, std::optional default_value) { @@ -603,148 +555,16 @@ Generator::Generator(const Poco::Util::AbstractConfiguration & config) acl.id = "anyone"; default_acls.emplace_back(std::move(acl)); - static const std::string generator_key = "generator"; - - std::cerr << "---- Parsing setup ---- " << std::endl; - static const std::string setup_key = generator_key + ".setup"; - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(setup_key, keys); - for (const auto & key : keys) - { - if (key.starts_with("node")) - { - auto node_key = setup_key + "." + key; - auto parsed_root_node = parseNode(node_key, config); - const auto node = root_nodes.emplace_back(parsed_root_node); - - if (config.has(node_key + ".repeat")) - { - if (!node->name.isRandom()) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Repeating node creation for key {}, but name is not randomly generated", node_key); - - auto repeat_count = config.getUInt64(node_key + ".repeat"); - node->repeat_count = repeat_count; - for (size_t i = 1; i < repeat_count; ++i) - root_nodes.emplace_back(node->clone()); - } - - std::cerr << "Tree to create:" << std::endl; - - node->dumpTree(); - std::cerr << std::endl; - } - } - std::cerr << "---- Done parsing data setup ----\n" << std::endl; - std::cerr << "---- Collecting request generators ----" << std::endl; - static const std::string requests_key = generator_key + ".requests"; + static const std::string requests_key = "generator.requests"; request_getter = RequestGetter::fromConfig(requests_key, config); std::cerr << request_getter.description() << std::endl; std::cerr << "---- Done collecting request generators ----\n" << std::endl; } -std::shared_ptr Generator::parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config) -{ - auto node = std::make_shared(); - node->name = StringGetter::fromConfig(key + ".name", config); - - if (config.has(key + ".data")) - node->data = StringGetter::fromConfig(key + ".data", config); - - Poco::Util::AbstractConfiguration::Keys node_keys; - config.keys(key, node_keys); - - for (const auto & node_key : node_keys) - { - if (!node_key.starts_with("node")) - continue; - - const auto node_key_string = key + "." + node_key; - auto child_node = parseNode(node_key_string, config); - node->children.push_back(child_node); - - if (config.has(node_key_string + ".repeat")) - { - if (!child_node->name.isRandom()) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Repeating node creation for key {}, but name is not randomly generated", node_key_string); - - auto repeat_count = config.getUInt64(node_key_string + ".repeat"); - child_node->repeat_count = repeat_count; - for (size_t i = 1; i < repeat_count; ++i) - node->children.push_back(child_node); - } - } - - return node; -} - -void Generator::Node::dumpTree(int level) const -{ - std::string data_string - = data.has_value() ? fmt::format("{}", data->description()) : "no data"; - - std::string repeat_count_string = repeat_count != 0 ? fmt::format(", repeated {} times", repeat_count) : ""; - - std::cerr << fmt::format("{}name: {}, data: {}{}", std::string(level, '\t'), name.description(), data_string, repeat_count_string) << std::endl; - - for (auto it = children.begin(); it != children.end();) - { - const auto & child = *it; - child->dumpTree(level + 1); - std::advance(it, child->repeat_count != 0 ? child->repeat_count : 1); - } -} - -std::shared_ptr Generator::Node::clone() const -{ - auto new_node = std::make_shared(); - new_node->name = name; - new_node->data = data; - new_node->repeat_count = repeat_count; - - // don't do deep copy of children because we will do clone only for root nodes - new_node->children = children; - - return new_node; -} - -void Generator::Node::createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const -{ - auto path = std::filesystem::path(parent_path) / name.getString(); - auto promise = std::make_shared>(); - auto future = promise->get_future(); - auto create_callback = [promise] (const CreateResponse & response) - { - if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(zkutil::KeeperException(response.error))); - else - promise->set_value(); - }; - zookeeper.create(path, data ? data->getString() : "", false, false, acls, create_callback); - future.get(); - - for (const auto & child : children) - child->createNode(zookeeper, path, acls); -} - void Generator::startup(Coordination::ZooKeeper & zookeeper) { - std::cerr << "---- Creating test data ----" << std::endl; - for (const auto & node : root_nodes) - { - auto node_name = node->name.getString(); - node->name.setString(node_name); - - std::string root_path = std::filesystem::path("/") / node_name; - std::cerr << "Cleaning up " << root_path << std::endl; - removeRecursive(zookeeper, root_path); - - node->createNode(zookeeper, "/", default_acls); - } - std::cerr << "---- Created test data ----\n" << std::endl; - std::cerr << "---- Initializing generators ----" << std::endl; - request_getter.startup(zookeeper); } @@ -752,15 +572,3 @@ Coordination::ZooKeeperRequestPtr Generator::generate() { return request_getter.getRequestGenerator()->generate(default_acls); } - -void Generator::cleanup(Coordination::ZooKeeper & zookeeper) -{ - std::cerr << "---- Cleaning up test data ----" << std::endl; - for (const auto & node : root_nodes) - { - auto node_name = node->name.getString(); - std::string root_path = std::filesystem::path("/") / node_name; - std::cerr << "Cleaning up " << root_path << std::endl; - removeRecursive(zookeeper, root_path); - } -} diff --git a/utils/keeper-bench/Generator.h b/utils/keeper-bench/Generator.h index 5b4c05b2d8b..35dce1a95d9 100644 --- a/utils/keeper-bench/Generator.h +++ b/utils/keeper-bench/Generator.h @@ -173,27 +173,9 @@ public: void startup(Coordination::ZooKeeper & zookeeper); Coordination::ZooKeeperRequestPtr generate(); - void cleanup(Coordination::ZooKeeper & zookeeper); private: - struct Node - { - StringGetter name; - std::optional data; - std::vector> children; - size_t repeat_count = 0; - - std::shared_ptr clone() const; - - void createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const; - void dumpTree(int level = 0) const; - }; - - static std::shared_ptr parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config); std::uniform_int_distribution request_picker; - std::vector> root_nodes; RequestGetter request_getter; Coordination::ACLs default_acls; }; - -std::optional getGenerator(const std::string & name); diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index a4b579f1f7b..ed7e09685f0 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -1,14 +1,31 @@ #include "Runner.h" +#include #include -#include "Common/ZooKeeper/ZooKeeperCommon.h" -#include "Common/ZooKeeper/ZooKeeperConstants.h" -#include -#include -#include "IO/ReadBufferFromString.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace CurrentMetrics @@ -22,23 +39,43 @@ namespace DB::ErrorCodes { extern const int CANNOT_BLOCK_SIGNAL; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } Runner::Runner( std::optional concurrency_, const std::string & config_path, + const std::string & input_request_log_, + const std::string & setup_nodes_snapshot_path_, const Strings & hosts_strings_, std::optional max_time_, std::optional delay_, std::optional continue_on_error_, std::optional max_iterations_) - : info(std::make_shared()) + : input_request_log(input_request_log_) + , setup_nodes_snapshot_path(setup_nodes_snapshot_path_) + , info(std::make_shared()) { DB::ConfigProcessor config_processor(config_path, true, false); - auto config = config_processor.loadConfig().configuration; + DB::ConfigurationPtr config = nullptr; + + if (!config_path.empty()) + { + config = config_processor.loadConfig().configuration; + + if (config->has("generator")) + generator.emplace(*config); + } + else + { + if (input_request_log.empty()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Both --config and --input_request_log cannot be empty"); + + if (!std::filesystem::exists(input_request_log)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "File on path {} does not exist", input_request_log); + } - generator.emplace(*config); if (!hosts_strings_.empty()) { @@ -57,6 +94,8 @@ Runner::Runner( static constexpr uint64_t DEFAULT_CONCURRENCY = 1; if (concurrency_) concurrency = *concurrency_; + else if (!config) + concurrency = DEFAULT_CONCURRENCY; else concurrency = config->getUInt64("concurrency", DEFAULT_CONCURRENCY); std::cerr << "Concurrency: " << concurrency << std::endl; @@ -64,6 +103,8 @@ Runner::Runner( static constexpr uint64_t DEFAULT_ITERATIONS = 0; if (max_iterations_) max_iterations = *max_iterations_; + else if (!config) + max_iterations = DEFAULT_ITERATIONS; else max_iterations = config->getUInt64("iterations", DEFAULT_ITERATIONS); std::cerr << "Iterations: " << max_iterations << std::endl; @@ -71,6 +112,8 @@ Runner::Runner( static constexpr double DEFAULT_DELAY = 1.0; if (delay_) delay = *delay_; + else if (!config) + delay = DEFAULT_DELAY; else delay = config->getDouble("report_delay", DEFAULT_DELAY); std::cerr << "Report delay: " << delay << std::endl; @@ -78,44 +121,48 @@ Runner::Runner( static constexpr double DEFAULT_TIME_LIMIT = 0.0; if (max_time_) max_time = *max_time_; + else if (!config) + max_time = DEFAULT_TIME_LIMIT; else max_time = config->getDouble("timelimit", DEFAULT_TIME_LIMIT); std::cerr << "Time limit: " << max_time << std::endl; if (continue_on_error_) continue_on_error = *continue_on_error_; + else if (!config) + continue_on_error_ = false; else continue_on_error = config->getBool("continue_on_error", false); std::cerr << "Continue on error: " << continue_on_error << std::endl; - static const std::string output_key = "output"; - print_to_stdout = config->getBool(output_key + ".stdout", false); - std::cerr << "Printing output to stdout: " << print_to_stdout << std::endl; - - static const std::string output_file_key = output_key + ".file"; - if (config->has(output_file_key)) + if (config) { - if (config->has(output_file_key + ".path")) - { - file_output = config->getString(output_file_key + ".path"); - output_file_with_timestamp = config->getBool(output_file_key + ".with_timestamp"); - } - else - file_output = config->getString(output_file_key); + benchmark_context.initializeFromConfig(*config); - std::cerr << "Result file path: " << file_output->string() << std::endl; + static const std::string output_key = "output"; + print_to_stdout = config->getBool(output_key + ".stdout", false); + std::cerr << "Printing output to stdout: " << print_to_stdout << std::endl; + + static const std::string output_file_key = output_key + ".file"; + if (config->has(output_file_key)) + { + if (config->has(output_file_key + ".path")) + { + file_output = config->getString(output_file_key + ".path"); + output_file_with_timestamp = config->getBool(output_file_key + ".with_timestamp"); + } + else + file_output = config->getString(output_file_key); + + std::cerr << "Result file path: " << file_output->string() << std::endl; + } } std::cerr << "---- Run options ----\n" << std::endl; - - pool.emplace(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, concurrency); - queue.emplace(concurrency); } void Runner::parseHostsFromConfig(const Poco::Util::AbstractConfiguration & config) { - ConnectionInfo default_connection_info; - const auto fill_connection_details = [&](const std::string & key, auto & connection_info) { if (config.has(key + ".secure")) @@ -328,9 +375,770 @@ bool Runner::tryPushRequestInteractively(Coordination::ZooKeeperRequestPtr && re void Runner::runBenchmark() { + if (generator) + runBenchmarkWithGenerator(); + else + runBenchmarkFromLog(); +} + + +struct ZooKeeperRequestBlock +{ + explicit ZooKeeperRequestBlock(DB::Block block_) + : block(std::move(block_)) + , hostname_idx(block.getPositionByName("hostname")) + , request_event_time_idx(block.getPositionByName("request_event_time")) + , thread_id_idx(block.getPositionByName("thread_id")) + , session_id_idx(block.getPositionByName("session_id")) + , xid_idx(block.getPositionByName("xid")) + , has_watch_idx(block.getPositionByName("has_watch")) + , op_num_idx(block.getPositionByName("op_num")) + , path_idx(block.getPositionByName("path")) + , data_idx(block.getPositionByName("data")) + , is_ephemeral_idx(block.getPositionByName("is_ephemeral")) + , is_sequential_idx(block.getPositionByName("is_sequential")) + , response_event_time_idx(block.getPositionByName("response_event_time")) + , error_idx(block.getPositionByName("error")) + , requests_size_idx(block.getPositionByName("requests_size")) + , version_idx(block.getPositionByName("version")) + {} + + size_t rows() const + { + return block.rows(); + } + + UInt64 getExecutorId(size_t row) const + { + return getSessionId(row); + } + + std::string getHostname(size_t row) const + { + return getField(hostname_idx, row).safeGet(); + } + + UInt64 getThreadId(size_t row) const + { + return getField(thread_id_idx, row).safeGet(); + } + + DB::DateTime64 getRequestEventTime(size_t row) const + { + return getField(request_event_time_idx, row).safeGet(); + } + + DB::DateTime64 getResponseEventTime(size_t row) const + { + return getField(response_event_time_idx, row).safeGet(); + } + + Int64 getSessionId(size_t row) const + { + return getField(session_id_idx, row).safeGet(); + } + + Int64 getXid(size_t row) const + { + return getField(xid_idx, row).safeGet(); + } + + bool hasWatch(size_t row) const + { + return getField(has_watch_idx, row).safeGet(); + } + + Coordination::OpNum getOpNum(size_t row) const + { + return static_cast(getField(op_num_idx, row).safeGet()); + } + + bool isEphemeral(size_t row) const + { + return getField(is_ephemeral_idx, row).safeGet(); + } + + bool isSequential(size_t row) const + { + return getField(is_sequential_idx, row).safeGet(); + } + + std::string getPath(size_t row) const + { + return getField(path_idx, row).safeGet(); + } + + std::string getData(size_t row) const + { + return getField(data_idx, row).safeGet(); + } + + UInt64 getRequestsSize(size_t row) const + { + return getField(requests_size_idx, row).safeGet(); + } + + std::optional getVersion(size_t row) const + { + auto field = getField(version_idx, row); + if (field.isNull()) + return std::nullopt; + return static_cast(field.safeGet()); + } + + std::optional getError(size_t row) const + { + auto field = getField(error_idx, row); + if (field.isNull()) + return std::nullopt; + + return static_cast(field.safeGet()); + } +private: + DB::Field getField(size_t position, size_t row) const + { + DB::Field field; + block.getByPosition(position).column->get(row, field); + return field; + } + + DB::Block block; + size_t hostname_idx = 0; + size_t request_event_time_idx = 0; + size_t thread_id_idx = 0; + size_t session_id_idx = 0; + size_t xid_idx = 0; + size_t has_watch_idx = 0; + size_t op_num_idx = 0; + size_t path_idx = 0; + size_t data_idx = 0; + size_t is_ephemeral_idx = 0; + size_t is_sequential_idx = 0; + size_t response_event_time_idx = 0; + size_t error_idx = 0; + size_t requests_size_idx = 0; + size_t version_idx = 0; +}; + +struct RequestFromLog +{ + Coordination::ZooKeeperRequestPtr request; + std::optional expected_result; + std::vector> subrequest_expected_results; + int64_t session_id = 0; + size_t executor_id = 0; + bool has_watch = false; + DB::DateTime64 request_event_time; + DB::DateTime64 response_event_time; + std::shared_ptr connection; +}; + +struct ZooKeeperRequestFromLogReader +{ + ZooKeeperRequestFromLogReader(const std::string & input_request_log, DB::ContextPtr context) + { + std::optional format_settings; + + file_read_buf = std::make_unique(input_request_log); + auto compression_method = DB::chooseCompressionMethod(input_request_log, ""); + file_read_buf = DB::wrapReadBufferWithCompressionMethod(std::move(file_read_buf), compression_method); + + DB::SingleReadBufferIterator read_buffer_iterator(std::move(file_read_buf)); + auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + + DB::ColumnsWithTypeAndName columns; + columns.reserve(columns_description.size()); + + for (const auto & column_description : columns_description) + columns.push_back(DB::ColumnWithTypeAndName{column_description.type, column_description.name}); + + header_block = std::move(columns); + + file_read_buf + = DB::wrapReadBufferWithCompressionMethod(std::make_unique(input_request_log), compression_method); + + input_format = DB::FormatFactory::instance().getInput( + format, + *file_read_buf, + header_block, + context, + context->getSettingsRef().max_block_size, + format_settings, + 1, + std::nullopt, + /*is_remote_fs*/ false, + DB::CompressionMethod::None, + false); + + Coordination::ACL acl; + acl.permissions = Coordination::ACL::All; + acl.scheme = "world"; + acl.id = "anyone"; + default_acls.emplace_back(std::move(acl)); + } + + std::optional getNextRequest(bool for_multi = false) + { + RequestFromLog request_from_log; + + if (!current_block) + { + auto chunk = input_format->generate(); + + if (chunk.empty()) + return std::nullopt; + + current_block.emplace(header_block.cloneWithColumns(chunk.detachColumns())); + idx_in_block = 0; + } + + request_from_log.expected_result = current_block->getError(idx_in_block); + request_from_log.session_id = current_block->getSessionId(idx_in_block); + request_from_log.has_watch = current_block->hasWatch(idx_in_block); + request_from_log.executor_id = current_block->getExecutorId(idx_in_block); + request_from_log.request_event_time = current_block->getRequestEventTime(idx_in_block); + request_from_log.response_event_time = current_block->getResponseEventTime(idx_in_block); + + const auto move_row_iterator = [&] + { + if (idx_in_block == current_block->rows() - 1) + current_block.reset(); + else + ++idx_in_block; + }; + + auto op_num = current_block->getOpNum(idx_in_block); + switch (op_num) + { + case Coordination::OpNum::Create: + { + auto create_request = std::make_shared(); + create_request->path = current_block->getPath(idx_in_block); + create_request->data = current_block->getData(idx_in_block); + create_request->is_ephemeral = current_block->isEphemeral(idx_in_block); + create_request->is_sequential = current_block->isSequential(idx_in_block); + request_from_log.request = create_request; + break; + } + case Coordination::OpNum::Set: + { + auto set_request = std::make_shared(); + set_request->path = current_block->getPath(idx_in_block); + set_request->data = current_block->getData(idx_in_block); + if (auto version = current_block->getVersion(idx_in_block)) + { + /// we just need to make sure that the request with version that need to fail, fail when replaying + if (request_from_log.expected_result == Coordination::Error::ZBADVERSION) + set_request->version = std::numeric_limits::max(); + } + request_from_log.request = set_request; + break; + } + case Coordination::OpNum::Remove: + { + auto remove_request = std::make_shared(); + remove_request->path = current_block->getPath(idx_in_block); + if (auto version = current_block->getVersion(idx_in_block)) + { + /// we just need to make sure that the request with version that need to fail, fail when replaying + if (request_from_log.expected_result == Coordination::Error::ZBADVERSION) + remove_request->version = std::numeric_limits::max(); + } + request_from_log.request = remove_request; + break; + } + case Coordination::OpNum::Check: + case Coordination::OpNum::CheckNotExists: + { + auto check_request = std::make_shared(); + check_request->path = current_block->getPath(idx_in_block); + if (auto version = current_block->getVersion(idx_in_block)) + { + /// we just need to make sure that the request with version that need to fail, fail when replaying + if (request_from_log.expected_result == Coordination::Error::ZBADVERSION) + check_request->version = std::numeric_limits::max(); + } + if (op_num == Coordination::OpNum::CheckNotExists) + check_request->not_exists = true; + request_from_log.request = check_request; + break; + } + case Coordination::OpNum::Sync: + { + auto sync_request = std::make_shared(); + sync_request->path = current_block->getPath(idx_in_block); + request_from_log.request = sync_request; + break; + } + case Coordination::OpNum::Get: + { + auto get_request = std::make_shared(); + get_request->path = current_block->getPath(idx_in_block); + request_from_log.request = get_request; + break; + } + case Coordination::OpNum::SimpleList: + case Coordination::OpNum::FilteredList: + { + auto list_request = std::make_shared(); + list_request->path = current_block->getPath(idx_in_block); + request_from_log.request = list_request; + break; + } + case Coordination::OpNum::Exists: + { + auto exists_request = std::make_shared(); + exists_request->path = current_block->getPath(idx_in_block); + request_from_log.request = exists_request; + break; + } + case Coordination::OpNum::Multi: + case Coordination::OpNum::MultiRead: + { + if (for_multi) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Nested multi requests are not allowed"); + + auto requests_size = current_block->getRequestsSize(idx_in_block); + + Coordination::Requests requests; + requests.reserve(requests_size); + move_row_iterator(); + + for (size_t i = 0; i < requests_size; ++i) + { + auto subrequest_from_log = getNextRequest(/*for_multi=*/true); + if (!subrequest_from_log) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Failed to fetch subrequest for {}, subrequest index {}", op_num, i); + + if (!subrequest_from_log->expected_result && request_from_log.expected_result + && request_from_log.expected_result == Coordination::Error::ZOK) + { + subrequest_from_log->expected_result = Coordination::Error::ZOK; + } + + requests.push_back(std::move(subrequest_from_log->request)); + + if (subrequest_from_log->session_id != request_from_log.session_id) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Session id mismatch for subrequest in {}, subrequest index {}", op_num, i); + + if (subrequest_from_log->executor_id != request_from_log.executor_id) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Executor id mismatch for subrequest in {}, subrequest index {}", op_num, i); + + request_from_log.subrequest_expected_results.push_back(subrequest_from_log->expected_result); + } + + request_from_log.request = std::make_shared(requests, default_acls); + + return request_from_log; + } + default: + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unsupported operation {} ({})", op_num, static_cast(op_num)); + } + + move_row_iterator(); + + return request_from_log; + } + +private: + DB::Block header_block; + + std::unique_ptr file_read_buf; + DB::InputFormatPtr input_format; + + std::optional current_block; + size_t idx_in_block = 0; + + Coordination::ACLs default_acls; +}; + + +namespace +{ + +struct RequestFromLogStats +{ + struct Stats + { + std::atomic total = 0; + std::atomic unexpected_results = 0; + }; + + Stats write_requests; + Stats read_requests; +}; + +struct SetupNodeCollector +{ + explicit SetupNodeCollector(const std::string & setup_nodes_snapshot_path) + { + if (setup_nodes_snapshot_path.empty()) + return; + + keeper_context = std::make_shared(true, std::make_shared()); + keeper_context->setDigestEnabled(true); + keeper_context->setSnapshotDisk( + std::make_shared("Keeper-snapshots", setup_nodes_snapshot_path)); + + snapshot_manager.emplace(1, keeper_context); + auto snapshot_result = snapshot_manager->restoreFromLatestSnapshot(); + if (snapshot_result.storage == nullptr) + { + std::cerr << "No initial snapshot found" << std::endl; + initial_storage = std::make_unique( + /* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); + initial_storage->initializeSystemNodes(); + } + else + { + std::cerr << "Loaded initial nodes from snapshot" << std::endl; + initial_storage = std::move(snapshot_result.storage); + } + } + + void processRequest(const RequestFromLog & request_from_log) + { + if (!request_from_log.expected_result.has_value()) + return; + + + auto process_request = [&](const Coordination::ZooKeeperRequest & request, const auto expected_result) + { + const auto & path = request.getPath(); + + if (nodes_created_during_replay.contains(path)) + return; + + auto op_num = request.getOpNum(); + + if (op_num == Coordination::OpNum::Create) + { + if (expected_result == Coordination::Error::ZNODEEXISTS) + { + addExpectedNode(path); + } + else if (expected_result == Coordination::Error::ZOK) + { + nodes_created_during_replay.insert(path); + /// we need to make sure ancestors exist + auto position = path.find_last_of('/'); + if (position != 0) + { + auto parent_path = path.substr(0, position); + addExpectedNode(parent_path); + } + } + } + else if (op_num == Coordination::OpNum::Remove) + { + if (expected_result == Coordination::Error::ZOK || expected_result == Coordination::Error::ZBADVERSION) + addExpectedNode(path); + } + else if (op_num == Coordination::OpNum::Set) + { + if (expected_result == Coordination::Error::ZOK || expected_result == Coordination::Error::ZBADVERSION) + addExpectedNode(path); + } + else if (op_num == Coordination::OpNum::Check) + { + if (expected_result == Coordination::Error::ZOK || expected_result == Coordination::Error::ZBADVERSION) + addExpectedNode(path); + } + else if (op_num == Coordination::OpNum::CheckNotExists) + { + if (expected_result == Coordination::Error::ZNODEEXISTS || expected_result == Coordination::Error::ZBADVERSION) + addExpectedNode(path); + } + else if (request.isReadRequest()) + { + if (expected_result == Coordination::Error::ZOK) + addExpectedNode(path); + } + }; + + const auto & request = request_from_log.request; + if (request->getOpNum() == Coordination::OpNum::Multi || request->getOpNum() == Coordination::OpNum::MultiRead) + { + const auto & multi_request = dynamic_cast(*request); + const auto & subrequests = multi_request.requests; + + for (size_t i = 0; i < subrequests.size(); ++i) + { + const auto & zookeeper_request = dynamic_cast(*subrequests[i]); + const auto subrequest_expected_result = request_from_log.subrequest_expected_results[i]; + if (subrequest_expected_result.has_value()) + process_request(zookeeper_request, *subrequest_expected_result); + + } + } + else + process_request(*request, *request_from_log.expected_result); + } + + void addExpectedNode(const std::string & path) + { + std::lock_guard lock(nodes_mutex); + + if (initial_storage->container.contains(path)) + return; + + new_nodes = true; + std::cerr << "Adding expected node " << path << std::endl; + + Coordination::Requests create_ops; + + size_t pos = 1; + while (true) + { + pos = path.find('/', pos); + if (pos == std::string::npos) + break; + + auto request = zkutil::makeCreateRequest(path.substr(0, pos), "", zkutil::CreateMode::Persistent, true); + create_ops.emplace_back(request); + ++pos; + } + + auto request = zkutil::makeCreateRequest(path, "", zkutil::CreateMode::Persistent, true); + create_ops.emplace_back(request); + + auto next_zxid = initial_storage->getNextZXID(); + + static Coordination::ACLs default_acls = [] + { + Coordination::ACL acl; + acl.permissions = Coordination::ACL::All; + acl.scheme = "world"; + acl.id = "anyone"; + return Coordination::ACLs{std::move(acl)}; + }(); + + auto multi_create_request = std::make_shared(create_ops, default_acls); + initial_storage->preprocessRequest(multi_create_request, 1, 0, next_zxid, /* check_acl = */ false); + auto responses = initial_storage->processRequest(multi_create_request, 1, next_zxid, /* check_acl = */ false); + if (responses.size() > 1 || responses[0].response->error != Coordination::Error::ZOK) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Invalid response after trying to create a node {}", responses[0].response->error); + } + + void generateSnapshot() + { + std::lock_guard lock(nodes_mutex); + if (!new_nodes) + { + std::cerr << "No new nodes added" << std::endl; + return; + } + + std::cerr << "Generating snapshot with starting data" << std::endl; + DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(initial_storage->getZXID(), 1, std::make_shared()); + DB::KeeperStorageSnapshot snapshot(initial_storage.get(), snapshot_meta); + snapshot_manager->serializeSnapshotToDisk(snapshot); + + new_nodes = false; + } + + std::mutex nodes_mutex; + DB::KeeperContextPtr keeper_context; + Coordination::KeeperStoragePtr initial_storage; + std::unordered_set nodes_created_during_replay; + std::optional snapshot_manager; + bool new_nodes = false; +}; + +void dumpStats(std::string_view type, const RequestFromLogStats::Stats & stats_for_type) +{ + std::cerr << fmt::format( + "{} requests: {} total, {} with unexpected results ({:.4}%)", + type, + stats_for_type.total, + stats_for_type.unexpected_results, + stats_for_type.total != 0 ? static_cast(stats_for_type.unexpected_results) / stats_for_type.total * 100 : 0.0) + << std::endl; +}; + +void requestFromLogExecutor(std::shared_ptr> queue, RequestFromLogStats & request_stats) +{ + RequestFromLog request_from_log; + std::optional> last_request; + while (queue->pop(request_from_log)) + { + auto request_promise = std::make_shared>(); + last_request = request_promise->get_future(); + Coordination::ResponseCallback callback = [&, + request_promise, + request = request_from_log.request, + expected_result = request_from_log.expected_result, + subrequest_expected_results = std::move(request_from_log.subrequest_expected_results)]( + const Coordination::Response & response) mutable + { + auto & stats = request->isReadRequest() ? request_stats.read_requests : request_stats.write_requests; + + stats.total.fetch_add(1, std::memory_order_relaxed); + + if (expected_result) + { + if (*expected_result != response.error) + stats.unexpected_results.fetch_add(1, std::memory_order_relaxed); + +#if 0 + if (*expected_result != response.error) + { + std::cerr << fmt::format( + "Unexpected result for {}\ngot {}, expected {}\n", request->toString(), response.error, *expected_result) + << std::endl; + + if (const auto * multi_response = dynamic_cast(&response)) + { + std::string subresponses; + for (size_t i = 0; i < multi_response->responses.size(); ++i) + { + subresponses += fmt::format("{} = {}\n", i, multi_response->responses[i]->error); + } + + std::cerr << "Subresponses\n" << subresponses << std::endl; + } + } +#endif + } + + request_promise->set_value(); + }; + + Coordination::WatchCallbackPtr watch; + if (request_from_log.has_watch) + watch = std::make_shared([](const Coordination::WatchResponse &) {}); + + request_from_log.connection->executeGenericRequest(request_from_log.request, callback, watch); + } + + if (last_request) + last_request->wait(); +} + +} + +void Runner::runBenchmarkFromLog() +{ + std::cerr << fmt::format("Running benchmark using requests from {}", input_request_log) << std::endl; + + pool.emplace(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, concurrency); + + shared_context = DB::Context::createShared(); + global_context = DB::Context::createGlobal(shared_context.get()); + global_context->makeGlobalContext(); + DB::registerFormats(); + + /// Randomly choosing connection index + pcg64 rng(randomSeed()); + std::uniform_int_distribution connection_distribution(0, connection_infos.size() - 1); + + std::unordered_map> zookeeper_connections; + auto get_zookeeper_connection = [&](int64_t session_id) + { + if (auto it = zookeeper_connections.find(session_id); it != zookeeper_connections.end() && !it->second->isExpired()) + return it->second; + + auto connection_idx = connection_distribution(rng); + auto zk_connection = getConnection(connection_infos[connection_idx], connection_idx); + zookeeper_connections.insert_or_assign(session_id, zk_connection); + return zk_connection; + }; + + RequestFromLogStats stats; + + std::optional setup_nodes_collector; + if (!setup_nodes_snapshot_path.empty()) + setup_nodes_collector.emplace(setup_nodes_snapshot_path); + + std::unordered_map>> executor_id_to_queue; + + SCOPE_EXIT_SAFE({ + for (const auto & [executor_id, executor_queue] : executor_id_to_queue) + executor_queue->finish(); + + pool->wait(); + + + if (setup_nodes_collector) + { + setup_nodes_collector->generateSnapshot(); + } + else + { + dumpStats("Write", stats.write_requests); + dumpStats("Read", stats.read_requests); + } + }); + + auto push_request = [&](RequestFromLog request) + { + if (auto it = executor_id_to_queue.find(request.executor_id); it != executor_id_to_queue.end()) + { + auto success = it->second->push(std::move(request)); + if (!success) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to push to the executor's queue"); + return; + } + + auto executor_queue = std::make_shared>(std::numeric_limits::max()); + executor_id_to_queue.emplace(request.executor_id, executor_queue); + auto scheduled = pool->trySchedule([&, executor_queue]() mutable + { + requestFromLogExecutor(std::move(executor_queue), stats); + }); + + if (!scheduled) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Failed to schedule worker, try to increase concurrency parameter"); + + auto success = executor_queue->push(std::move(request)); + if (!success) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to push to the executor's queue"); + }; + + if (!setup_nodes_collector) + { + auto setup_connection = getConnection(connection_infos[0], 0); + benchmark_context.startup(*setup_connection); + } + + ZooKeeperRequestFromLogReader request_reader(input_request_log, global_context); + + delay_watch.restart(); + while (auto request_from_log = request_reader.getNextRequest()) + { + if (setup_nodes_collector) + { + setup_nodes_collector->processRequest(*request_from_log); + } + else + { + request_from_log->connection = get_zookeeper_connection(request_from_log->session_id); + push_request(std::move(*request_from_log)); + } + + if (delay > 0 && delay_watch.elapsedSeconds() > delay) + { + if (setup_nodes_collector) + setup_nodes_collector->generateSnapshot(); + else + { + dumpStats("Write", stats.write_requests); + dumpStats("Read", stats.read_requests); + std::cerr << std::endl; + } + delay_watch.restart(); + } + } +} + +void Runner::runBenchmarkWithGenerator() +{ + pool.emplace(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, concurrency); + queue.emplace(concurrency); createConnections(); std::cerr << "Preparing to run\n"; + benchmark_context.startup(*connections[0]); generator->startup(*connections[0]); std::cerr << "Prepared\n"; @@ -341,7 +1149,7 @@ void Runner::runBenchmark() for (size_t i = 0; i < concurrency; ++i) { auto thread_connections = connections; - pool->scheduleOrThrowOnError([this, connections_ = std::move(thread_connections)]() mutable { thread(connections_); }); + pool->scheduleOrThrowOnError([this, my_connections = std::move(thread_connections)]() mutable { thread(my_connections); }); } } catch (...) @@ -458,8 +1266,232 @@ std::vector> Runner::refreshConnections Runner::~Runner() { - queue->clearAndFinish(); + if (queue) + queue->clearAndFinish(); shutdown = true; - pool->wait(); - generator->cleanup(*connections[0]); + + if (pool) + pool->wait(); + + try + { + auto connection = getConnection(connection_infos[0], 0); + benchmark_context.cleanup(*connection); + } + catch (...) + { + DB::tryLogCurrentException("While trying to clean nodes"); + } +} + +namespace +{ + +void removeRecursive(Coordination::ZooKeeper & zookeeper, const std::string & path) +{ + namespace fs = std::filesystem; + + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + Strings children; + auto list_callback = [promise, &children] (const Coordination::ListResponse & response) + { + children = response.names; + promise->set_value(); + }; + zookeeper.list(path, Coordination::ListRequestType::ALL, list_callback, nullptr); + future.get(); + + std::span children_span(children); + while (!children_span.empty()) + { + Coordination::Requests ops; + for (size_t i = 0; i < 1000 && !children.empty(); ++i) + { + removeRecursive(zookeeper, fs::path(path) / children.back()); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(path) / children_span.back(), -1)); + children_span = children_span.subspan(0, children_span.size() - 1); + } + auto multi_promise = std::make_shared>(); + auto multi_future = multi_promise->get_future(); + + auto multi_callback = [multi_promise] (const Coordination::MultiResponse &) + { + multi_promise->set_value(); + }; + zookeeper.multi(ops, multi_callback); + multi_future.get(); + } + auto remove_promise = std::make_shared>(); + auto remove_future = remove_promise->get_future(); + + auto remove_callback = [remove_promise] (const Coordination::RemoveResponse &) + { + remove_promise->set_value(); + }; + + zookeeper.remove(path, -1, remove_callback); + remove_future.get(); +} + +} + +void BenchmarkContext::initializeFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + Coordination::ACL acl; + acl.permissions = Coordination::ACL::All; + acl.scheme = "world"; + acl.id = "anyone"; + default_acls.emplace_back(std::move(acl)); + + std::cerr << "---- Parsing setup ---- " << std::endl; + static const std::string setup_key = "setup"; + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(setup_key, keys); + for (const auto & key : keys) + { + if (key.starts_with("node")) + { + auto node_key = setup_key + "." + key; + auto parsed_root_node = parseNode(node_key, config); + const auto node = root_nodes.emplace_back(parsed_root_node); + + if (config.has(node_key + ".repeat")) + { + if (!node->name.isRandom()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Repeating node creation for key {}, but name is not randomly generated", node_key); + + auto repeat_count = config.getUInt64(node_key + ".repeat"); + node->repeat_count = repeat_count; + for (size_t i = 1; i < repeat_count; ++i) + root_nodes.emplace_back(node->clone()); + } + + std::cerr << "Tree to create:" << std::endl; + + node->dumpTree(); + std::cerr << std::endl; + } + } + std::cerr << "---- Done parsing data setup ----\n" << std::endl; +} + +std::shared_ptr BenchmarkContext::parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config) +{ + auto node = std::make_shared(); + node->name = StringGetter::fromConfig(key + ".name", config); + + if (config.has(key + ".data")) + node->data = StringGetter::fromConfig(key + ".data", config); + + Poco::Util::AbstractConfiguration::Keys node_keys; + config.keys(key, node_keys); + + for (const auto & node_key : node_keys) + { + if (!node_key.starts_with("node")) + continue; + + const auto node_key_string = key + "." + node_key; + auto child_node = parseNode(node_key_string, config); + node->children.push_back(child_node); + + if (config.has(node_key_string + ".repeat")) + { + if (!child_node->name.isRandom()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Repeating node creation for key {}, but name is not randomly generated", node_key_string); + + auto repeat_count = config.getUInt64(node_key_string + ".repeat"); + child_node->repeat_count = repeat_count; + for (size_t i = 1; i < repeat_count; ++i) + node->children.push_back(child_node); + } + } + + return node; +} + +void BenchmarkContext::Node::dumpTree(int level) const +{ + std::string data_string + = data.has_value() ? fmt::format("{}", data->description()) : "no data"; + + std::string repeat_count_string = repeat_count != 0 ? fmt::format(", repeated {} times", repeat_count) : ""; + + std::cerr << fmt::format("{}name: {}, data: {}{}", std::string(level, '\t'), name.description(), data_string, repeat_count_string) << std::endl; + + for (auto it = children.begin(); it != children.end();) + { + const auto & child = *it; + child->dumpTree(level + 1); + std::advance(it, child->repeat_count != 0 ? child->repeat_count : 1); + } +} + +std::shared_ptr BenchmarkContext::Node::clone() const +{ + auto new_node = std::make_shared(); + new_node->name = name; + new_node->data = data; + new_node->repeat_count = repeat_count; + + // don't do deep copy of children because we will do clone only for root nodes + new_node->children = children; + + return new_node; +} + +void BenchmarkContext::Node::createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const +{ + auto path = std::filesystem::path(parent_path) / name.getString(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); + auto create_callback = [promise] (const Coordination::CreateResponse & response) + { + if (response.error != Coordination::Error::ZOK) + promise->set_exception(std::make_exception_ptr(zkutil::KeeperException(response.error))); + else + promise->set_value(); + }; + zookeeper.create(path, data ? data->getString() : "", false, false, acls, create_callback); + future.get(); + + for (const auto & child : children) + child->createNode(zookeeper, path, acls); +} + +void BenchmarkContext::startup(Coordination::ZooKeeper & zookeeper) +{ + if (root_nodes.empty()) + return; + + std::cerr << "---- Creating test data ----" << std::endl; + for (const auto & node : root_nodes) + { + auto node_name = node->name.getString(); + node->name.setString(node_name); + + std::string root_path = std::filesystem::path("/") / node_name; + std::cerr << "Cleaning up " << root_path << std::endl; + removeRecursive(zookeeper, root_path); + + node->createNode(zookeeper, "/", default_acls); + } + std::cerr << "---- Created test data ----\n" << std::endl; +} + +void BenchmarkContext::cleanup(Coordination::ZooKeeper & zookeeper) +{ + if (root_nodes.empty()) + return; + + std::cerr << "---- Cleaning up test data ----" << std::endl; + for (const auto & node : root_nodes) + { + auto node_name = node->name.getString(); + std::string root_path = std::filesystem::path("/") / node_name; + std::cerr << "Cleaning up " << root_path << std::endl; + removeRecursive(zookeeper, root_path); + } } diff --git a/utils/keeper-bench/Runner.h b/utils/keeper-bench/Runner.h index 4f4a75e6ecf..c19a4d82898 100644 --- a/utils/keeper-bench/Runner.h +++ b/utils/keeper-bench/Runner.h @@ -1,5 +1,5 @@ #pragma once -#include "Common/ZooKeeper/ZooKeeperConstants.h" +#include "Common/ZooKeeper/ZooKeeperArgs.h" #include #include "Generator.h" #include @@ -12,6 +12,7 @@ #include #include +#include "Interpreters/Context.h" #include "Stats.h" #include @@ -19,12 +20,42 @@ using Ports = std::vector; using Strings = std::vector; +struct BenchmarkContext +{ +public: + void initializeFromConfig(const Poco::Util::AbstractConfiguration & config); + + void startup(Coordination::ZooKeeper & zookeeper); + void cleanup(Coordination::ZooKeeper & zookeeper); + +private: + struct Node + { + StringGetter name; + std::optional data; + std::vector> children; + size_t repeat_count = 0; + + std::shared_ptr clone() const; + + void createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const; + void dumpTree(int level = 0) const; + }; + + static std::shared_ptr parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config); + + std::vector> root_nodes; + Coordination::ACLs default_acls; +}; + class Runner { public: Runner( std::optional concurrency_, const std::string & config_path, + const std::string & input_request_log_, + const std::string & setup_nodes_snapshot_path_, const Strings & hosts_strings_, std::optional max_time_, std::optional delay_, @@ -44,8 +75,31 @@ public: ~Runner(); private: + struct ConnectionInfo + { + std::string host; + + bool secure = false; + int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; + int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; + int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; + bool use_compression = false; + + size_t sessions = 1; + }; + void parseHostsFromConfig(const Poco::Util::AbstractConfiguration & config); + void runBenchmarkWithGenerator(); + void runBenchmarkFromLog(); + + void createConnections(); + std::vector> refreshConnections(); + std::shared_ptr getConnection(const ConnectionInfo & connection_info, size_t connection_info_idx); + + std::string input_request_log; + std::string setup_nodes_snapshot_path; + size_t concurrency = 1; std::optional pool; @@ -54,7 +108,8 @@ private: double max_time = 0; double delay = 1; bool continue_on_error = false; - std::atomic max_iterations = 0; + size_t max_iterations = 0; + std::atomic requests_executed = 0; std::atomic shutdown = false; @@ -71,25 +126,14 @@ private: using Queue = ConcurrentBoundedQueue; std::optional queue; - struct ConnectionInfo - { - std::string host; - - bool secure = false; - int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; - int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; - int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; - bool use_compression = false; - - size_t sessions = 1; - }; - std::mutex connection_mutex; + ConnectionInfo default_connection_info; std::vector connection_infos; std::vector> connections; std::unordered_map connections_to_info_map; - void createConnections(); - std::shared_ptr getConnection(const ConnectionInfo & connection_info, size_t connection_info_idx); - std::vector> refreshConnections(); + DB::SharedContextHolder shared_context; + DB::ContextMutablePtr global_context; + + BenchmarkContext benchmark_context; }; diff --git a/utils/keeper-bench/main.cpp b/utils/keeper-bench/main.cpp index 0753d66850f..0b963abf406 100644 --- a/utils/keeper-bench/main.cpp +++ b/utils/keeper-bench/main.cpp @@ -1,8 +1,6 @@ #include #include #include "Runner.h" -#include "Stats.h" -#include "Generator.h" #include "Common/Exception.h" #include #include @@ -27,6 +25,10 @@ int main(int argc, char *argv[]) bool print_stacktrace = true; + //Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + //Poco::Logger::root().setChannel(channel); + //Poco::Logger::root().setLevel("trace"); + try { using boost::program_options::value; @@ -34,12 +36,14 @@ int main(int argc, char *argv[]) boost::program_options::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help", "produce help message") - ("config", value()->default_value(""), "yaml/xml file containing configuration") - ("concurrency,c", value(), "number of parallel queries") - ("report-delay,d", value(), "delay between intermediate reports in seconds (set 0 to disable reports)") - ("iterations,i", value(), "amount of queries to be executed") - ("time-limit,t", value(), "stop launch of queries after specified time limit") - ("hosts,h", value()->multitoken()->default_value(Strings{}, ""), "") + ("config", value()->default_value(""), "yaml/xml file containing configuration") + ("input-request-log", value()->default_value(""), "log of requests that will be replayed") + ("setup-nodes-snapshot-path", value()->default_value(""), "directory containing snapshots with starting state") + ("concurrency,c", value(), "number of parallel queries") + ("report-delay,d", value(), "delay between intermediate reports in seconds (set 0 to disable reports)") + ("iterations,i", value(), "amount of queries to be executed") + ("time-limit,t", value(), "stop launch of queries after specified time limit") + ("hosts,h", value()->multitoken()->default_value(Strings{}, ""), "") ("continue_on_errors", "continue testing even if a query fails") ; @@ -56,6 +60,8 @@ int main(int argc, char *argv[]) Runner runner(valueToOptional(options["concurrency"]), options["config"].as(), + options["input-request-log"].as(), + options["setup-nodes-snapshot-path"].as(), options["hosts"].as(), valueToOptional(options["time-limit"]), valueToOptional(options["report-delay"]), @@ -66,9 +72,9 @@ int main(int argc, char *argv[]) { runner.runBenchmark(); } - catch (const DB::Exception & e) + catch (...) { - std::cout << "Got exception while trying to run benchmark: " << e.message() << std::endl; + std::cout << "Got exception while trying to run benchmark: " << DB::getCurrentExceptionMessage(true) << std::endl; } return 0;