diff --git a/docs/en/engines/table-engines/integrations/azureBlobStorage.md b/docs/en/engines/table-engines/integrations/azureBlobStorage.md index 3df08ee2ffb..c6525121667 100644 --- a/docs/en/engines/table-engines/integrations/azureBlobStorage.md +++ b/docs/en/engines/table-engines/integrations/azureBlobStorage.md @@ -47,6 +47,12 @@ SELECT * FROM test_table; └──────┴───────┘ ``` +## Virtual columns {#virtual-columns} + +- `_path` — Path to the file. Type: `LowCardinalty(String)`. +- `_file` — Name of the file. Type: `LowCardinalty(String)`. +- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. + ## See also [Azure Blob Storage Table Function](/docs/en/sql-reference/table-functions/azureBlobStorage) diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index c677123a8d0..19221c256f9 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -230,8 +230,9 @@ libhdfs3 support HDFS namenode HA. ## Virtual Columns {#virtual-columns} -- `_path` — Path to the file. -- `_file` — Name of the file. +- `_path` — Path to the file. Type: `LowCardinalty(String)`. +- `_file` — Name of the file. Type: `LowCardinalty(String)`. +- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 2967a15494c..3144bdd32fa 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -142,8 +142,9 @@ Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Reading fr ## Virtual columns {#virtual-columns} -- `_path` — Path to the file. -- `_file` — Name of the file. +- `_path` — Path to the file. Type: `LowCardinalty(String)`. +- `_file` — Name of the file. Type: `LowCardinalty(String)`. +- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index 27945b30c03..6e3897398a5 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -87,12 +87,18 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64 - Indices - Replication -## PARTITION BY +## PARTITION BY {#partition-by} `PARTITION BY` — Optional. It is possible to create separate files by partitioning the data on a partition key. In most cases, you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression). For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. +## Virtual Columns {#virtual-columns} + +- `_path` — Path to the file. Type: `LowCardinalty(String)`. +- `_file` — Name of the file. Type: `LowCardinalty(String)`. +- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. + ## Settings {#settings} - [engine_file_empty_if_not_exists](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - allows to select empty data from a file that doesn't exist. Disabled by default. diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 5a5e1564180..f6183a779ae 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -103,6 +103,12 @@ SELECT * FROM url_engine_table For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. +## Virtual Columns {#virtual-columns} + +- `_path` — Path to the `URL`. Type: `LowCardinalty(String)`. +- `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. +- `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. + ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 59c92e1327e..1510489ce83 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -67,6 +67,12 @@ SELECT count(*) FROM azureBlobStorage('DefaultEndpointsProtocol=https;AccountNam └─────────┘ ``` +## Virtual Columns {#virtual-columns} + +- `_path` — Path to the file. Type: `LowCardinalty(String)`. +- `_file` — Name of the file. Type: `LowCardinalty(String)`. +- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. + **See Also** - [AzureBlobStorage Table Engine](/docs/en/engines/table-engines/integrations/azureBlobStorage.md) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index a871bdaafa9..ad1feb87c60 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -191,12 +191,13 @@ Query the total number of rows from all files `file002` inside any folder in dir SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt32'); ``` -## Virtual Columns +## Virtual Columns {#virtual-columns} -- `_path` — Path to the file. -- `_file` — Name of the file. +- `_path` — Path to the file. Type: `LowCardinalty(String)`. +- `_file` — Name of the file. Type: `LowCardinalty(String)`. +- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. -## Settings +## Settings {#settings} - [engine_file_empty_if_not_exists](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - allows to select empty data from a file that doesn't exist. Disabled by default. - [engine_file_truncate_on_insert](/docs/en/operations/settings/settings.md#engine-file-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 678470e9150..31780e30e8e 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -94,8 +94,9 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin ## Virtual Columns -- `_path` — Path to the file. -- `_file` — Name of the file. +- `_path` — Path to the file. Type: `LowCardinalty(String)`. +- `_file` — Name of the file. Type: `LowCardinalty(String)`. +- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index c80488df05e..dc11259c626 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -228,6 +228,12 @@ FROM s3( LIMIT 5; ``` +## Virtual Columns {#virtual-columns} + +- `_path` — Path to the file. Type: `LowCardinalty(String)`. +- `_file` — Name of the file. Type: `LowCardinalty(String)`. +- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the file size is unknown, the value is `NULL`. + ## Storage Settings {#storage-settings} - [s3_truncate_on_insert](/docs/en/operations/settings/settings.md#s3-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 859de86f019..4dc6e435b50 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -50,8 +50,9 @@ Character `|` inside patterns is used to specify failover addresses. They are it ## Virtual Columns -- `_path` — Path to the `URL`. -- `_file` — Resource name of the `URL`. +- `_path` — Path to the `URL`. Type: `LowCardinalty(String)`. +- `_file` — Resource name of the `URL`. Type: `LowCardinalty(String)`. +- `_size` — Size of the resource in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`. ## Storage Settings {#storage-settings} diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 7d845dac57d..a911862aa1c 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -238,7 +238,7 @@ StorageHDFS::StorageHDFS( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } namespace @@ -540,6 +540,7 @@ bool HDFSSource::initialize() } 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; @@ -613,7 +614,7 @@ Chunk HDFSSource::generate() if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, current_path); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, current_path, current_file_size); return chunk; } diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index ffbf4e93ff9..9e53f1bd87c 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -181,6 +181,7 @@ private: 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(); diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 83655b06cc8..bff22936e95 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -66,7 +66,7 @@ StorageHDFSCluster::StorageHDFSCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 1afd17edbe1..27bec039f96 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -210,7 +210,7 @@ Chunk StorageS3QueueSource::generate() file_status->processed_rows += chunk.getNumRows(); processed_rows_from_file += chunk.getNumRows(); - VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath()); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath(), reader.getKeyWithInfo().info->size); return chunk; } } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 41512267b89..95cd80e898f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -150,7 +150,8 @@ StorageS3Queue::StorageS3Queue( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); task = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 5e5162a0b31..0750d0c1403 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -494,7 +494,7 @@ StorageAzureBlob::StorageAzureBlob( for (const auto & key : configuration.blobs_paths) objects.emplace_back(key); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } void StorageAzureBlob::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) @@ -1011,7 +1011,11 @@ Chunk StorageAzureBlobSource::generate() if (const auto * input_format = reader.getInputFormat()) chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, fs::path(container) / reader.getRelativePath()); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( + chunk, + requested_virtual_columns, + fs::path(container) / reader.getRelativePath(), + reader.getRelativePathWithMetadata().metadata.size_bytes); return chunk; } diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index 7592a6e6acc..b8f95458379 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -57,7 +57,7 @@ StorageAzureBlobCluster::StorageAzureBlobCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } void StorageAzureBlobCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index dd527a50794..9769ea09d80 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -859,7 +859,7 @@ void StorageFile::setStorageMetadata(CommonArguments args) storage_metadata.setComment(args.comment); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } @@ -1149,6 +1149,7 @@ public: chassert(file_enumerator); current_path = fmt::format("{}::{}", archive_reader->getPath(), *filename_override); + current_file_size = file_enumerator->getFileInfo().uncompressed_size; if (need_only_count && tryGetCountFromCache(current_archive_stat)) continue; @@ -1177,6 +1178,7 @@ public: { struct stat file_stat; file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); + current_file_size = file_stat.st_size; if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) continue; @@ -1243,8 +1245,8 @@ public: progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); /// Enrich with virtual columns. - VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk( - chunk, requested_virtual_columns, current_path, filename_override.has_value() ? &filename_override.value() : nullptr); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( + chunk, requested_virtual_columns, current_path, current_file_size, filename_override.has_value() ? &filename_override.value() : nullptr); return chunk; } @@ -1305,6 +1307,7 @@ private: StorageSnapshotPtr storage_snapshot; FilesIteratorPtr files_iterator; String current_path; + std::optional current_file_size; struct stat current_archive_stat; std::optional filename_override; Block sample_block; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 194f9cb647f..39a4e057b4b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -865,7 +865,7 @@ Chunk StorageS3Source::generate() if (const auto * input_format = reader.getInputFormat()) chunk_size = reader.getInputFormat()->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath()); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath(), reader.getFileSize()); return chunk; } @@ -1143,7 +1143,7 @@ StorageS3::StorageS3( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } static std::shared_ptr createFileIterator( diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index d83022a9223..93515dab130 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -229,6 +229,7 @@ private: String getPath() const { return fs::path(bucket) / key_with_info->key; } const String & getFile() const { return key_with_info->key; } 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()); } diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 824dae6bc3e..702b1f14ae7 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -61,7 +61,7 @@ StorageS3Cluster::StorageS3Cluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } void StorageS3Cluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 724574e9515..a485215f72f 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -140,7 +140,7 @@ IStorageURLBase::IStorageURLBase( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } @@ -308,12 +308,10 @@ StorageURLSource::StorageURLSource( curr_uri = uri_and_buf.first; auto last_mod_time = uri_and_buf.second->tryGetLastModificationTime(); read_buf = std::move(uri_and_buf.second); + current_file_size = tryGetFileSizeFromReadBuffer(*read_buf); if (auto file_progress_callback = getContext()->getFileProgressCallback()) - { - size_t file_size = tryGetFileSizeFromReadBuffer(*read_buf).value_or(0); - file_progress_callback(FileProgress(0, file_size)); - } + file_progress_callback(FileProgress(0, current_file_size.value_or(0))); QueryPipelineBuilder builder; std::optional num_rows_from_cache = std::nullopt; @@ -401,7 +399,7 @@ Chunk StorageURLSource::generate() if (input_format) chunk_size = input_format->getApproxBytesReadForChunk(); progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, curr_uri.getPath()); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, curr_uri.getPath(), current_file_size); return chunk; } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 6306d16742e..f3758f92468 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -225,6 +225,7 @@ private: Block block_for_format; std::shared_ptr uri_iterator; Poco::URI curr_uri; + std::optional current_file_size; String format; const std::optional & format_settings; HTTPHeaderEntries headers; diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 5c2108bef33..c052e781877 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -69,7 +69,7 @@ StorageURLCluster::StorageURLCluster( storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); - virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); } void StorageURLCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 7aec5ce0d78..86ba7ee3a34 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -350,11 +350,12 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex } } -NamesAndTypesList getPathAndFileVirtualsForStorage(NamesAndTypesList storage_columns) +NamesAndTypesList getPathFileAndSizeVirtualsForStorage(NamesAndTypesList storage_columns) { auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, - {"_file", std::make_shared(std::make_shared())}}; + {"_file", std::make_shared(std::make_shared())}, + {"_size", makeNullable(std::make_shared())}}; default_virtuals.sort(); storage_columns.sort(); @@ -395,7 +396,10 @@ ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList Block block; for (const auto & column : virtual_columns) - block.insert({column.type->createColumn(), column.type, column.name}); + { + if (column.name == "_file" || column.name == "_path") + block.insert({column.type->createColumn(), column.type, column.name}); + } /// Create a block with one row to construct filter /// Append "idx" column as the filter result block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); @@ -409,7 +413,10 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const { Block block; for (const auto & column : virtual_columns) - block.insert({column.type->createColumn(), column.type, column.name}); + { + if (column.name == "_file" || column.name == "_path") + block.insert({column.type->createColumn(), column.type, column.name}); + } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); for (size_t i = 0; i != paths.size(); ++i) @@ -420,8 +427,8 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const return block.getByName("_idx").column; } -void addRequestedPathAndFileVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, const String * filename) +void addRequestedPathFileAndSizeVirtualsToChunk( + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename) { for (const auto & virtual_column : requested_virtual_columns) { @@ -442,6 +449,13 @@ void addRequestedPathAndFileVirtualsToChunk( chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), filename_from_path)->convertToFullColumnIfConst()); } } + else if (virtual_column.name == "_size") + { + if (size) + chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *size)->convertToFullColumnIfConst()); + else + chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst()); + } } } diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index d8d4b44b4ff..cd9eca34c04 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -50,7 +50,7 @@ auto extractSingleValueFromBlock(const Block & block, const String & name) return res; } -NamesAndTypesList getPathAndFileVirtualsForStorage(NamesAndTypesList storage_columns); +NamesAndTypesList getPathFileAndSizeVirtualsForStorage(NamesAndTypesList storage_columns); ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList & virtual_columns, const String & path_example, const ContextPtr & context); @@ -71,8 +71,8 @@ void filterByPathOrFile(std::vector & sources, const std::vector & pa sources = std::move(filtered_sources); } -void addRequestedPathAndFileVirtualsToChunk( - Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, const String * filename = nullptr); +void addRequestedPathFileAndSizeVirtualsToChunk( + Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, std::optional size, const String * filename = nullptr); } } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e0365f70e7f..96fff6b891f 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1156,3 +1156,37 @@ def test_filtering_by_file_or_path(cluster): ) assert int(result) == 1 + + +def test_size_virtual_column(cluster): + node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_size_virtual_column1.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", + ) + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_size_virtual_column2.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 11", + ) + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_size_virtual_column3.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 111", + ) + + result = azure_query( + node, + f"select _file, _size from azureBlobStorage('{storage_account_url}', 'cont', 'test_size_virtual_column*.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') " + f"order by _file", + ) + + assert ( + result + == "test_size_virtual_column1.tsv\t2\ntest_size_virtual_column2.tsv\t3\ntest_size_virtual_column3.tsv\t4\n" + ) diff --git a/tests/queries/0_stateless/02921_file_engine_size_virtual_column.reference b/tests/queries/0_stateless/02921_file_engine_size_virtual_column.reference new file mode 100644 index 00000000000..2f319dfb812 --- /dev/null +++ b/tests/queries/0_stateless/02921_file_engine_size_virtual_column.reference @@ -0,0 +1,12 @@ +2 +3 +4 +2 +3 +4 +2 +3 +4 +2 +3 +4 diff --git a/tests/queries/0_stateless/02921_file_engine_size_virtual_column.sh b/tests/queries/0_stateless/02921_file_engine_size_virtual_column.sh new file mode 100755 index 00000000000..5dd58ec0d7f --- /dev/null +++ b/tests/queries/0_stateless/02921_file_engine_size_virtual_column.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "1" > $CLICKHOUSE_TEST_UNIQUE_NAME.data1.tsv +echo "12" > $CLICKHOUSE_TEST_UNIQUE_NAME.data2.tsv +echo "123" > $CLICKHOUSE_TEST_UNIQUE_NAME.data3.tsv + +$CLICKHOUSE_LOCAL -q "select _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.data{1,2,3}.tsv') order by _size" +# Run this query twice to check correct behaviour when cache is used +$CLICKHOUSE_LOCAL -q "select _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.data{1,2,3}.tsv') order by _size" + +# Test the same fils in archive +tar -cf $CLICKHOUSE_TEST_UNIQUE_NAME.archive.tar $CLICKHOUSE_TEST_UNIQUE_NAME.data1.tsv $CLICKHOUSE_TEST_UNIQUE_NAME.data2.tsv $CLICKHOUSE_TEST_UNIQUE_NAME.data3.tsv + +$CLICKHOUSE_LOCAL -q "select _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.archive.tar :: $CLICKHOUSE_TEST_UNIQUE_NAME.data{1,2,3}.tsv') order by _size" +$CLICKHOUSE_LOCAL -q "select _size from file('$CLICKHOUSE_TEST_UNIQUE_NAME.archive.tar :: $CLICKHOUSE_TEST_UNIQUE_NAME.data{1,2,3}.tsv') order by _size" + +rm $CLICKHOUSE_TEST_UNIQUE_NAME.* + diff --git a/tests/queries/0_stateless/02922_url_s3_engine_size_virtual_column.reference b/tests/queries/0_stateless/02922_url_s3_engine_size_virtual_column.reference new file mode 100644 index 00000000000..369837adcbb --- /dev/null +++ b/tests/queries/0_stateless/02922_url_s3_engine_size_virtual_column.reference @@ -0,0 +1,12 @@ +a.tsv 24 +b.tsv 33 +c.tsv 33 +a.tsv 24 +b.tsv 33 +c.tsv 33 +a.tsv 24 +b.tsv 33 +c.tsv 33 +a.tsv 24 +b.tsv 33 +c.tsv 33 diff --git a/tests/queries/0_stateless/02922_url_s3_engine_size_virtual_column.sh b/tests/queries/0_stateless/02922_url_s3_engine_size_virtual_column.sh new file mode 100755 index 00000000000..51de2117dca --- /dev/null +++ b/tests/queries/0_stateless/02922_url_s3_engine_size_virtual_column.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "select _file, _size from url('http://localhost:11111/test/{a,b,c}.tsv', 'One') order by _file" +$CLICKHOUSE_CLIENT -q "select _file, _size from url('http://localhost:11111/test/{a,b,c}.tsv', 'One') order by _file" + +$CLICKHOUSE_CLIENT -q "select _file, _size from s3('http://localhost:11111/test/{a,b,c}.tsv', 'One') order by _file" +$CLICKHOUSE_CLIENT -q "select _file, _size from s3('http://localhost:11111/test/{a,b,c}.tsv', 'One') order by _file" + diff --git a/tests/queries/0_stateless/02923_hdfs_engine_size_virtual_column.reference b/tests/queries/0_stateless/02923_hdfs_engine_size_virtual_column.reference new file mode 100644 index 00000000000..bc42121fb39 --- /dev/null +++ b/tests/queries/0_stateless/02923_hdfs_engine_size_virtual_column.reference @@ -0,0 +1,6 @@ +2 +3 +4 +2 +3 +4 diff --git a/tests/queries/0_stateless/02923_hdfs_engine_size_virtual_column.sh b/tests/queries/0_stateless/02923_hdfs_engine_size_virtual_column.sh new file mode 100755 index 00000000000..dc01687772f --- /dev/null +++ b/tests/queries/0_stateless/02923_hdfs_engine_size_virtual_column.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, use-hdfs + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "insert into table function hdfs('hdfs://localhost:12222/$CLICKHOUSE_TEST_UNIQUE_NAME.data1.tsv') select 1 settings hdfs_truncate_on_insert=1;" +$CLICKHOUSE_CLIENT -q "insert into table function hdfs('hdfs://localhost:12222/$CLICKHOUSE_TEST_UNIQUE_NAME.data2.tsv') select 11 settings hdfs_truncate_on_insert=1;" +$CLICKHOUSE_CLIENT -q "insert into table function hdfs('hdfs://localhost:12222/$CLICKHOUSE_TEST_UNIQUE_NAME.data3.tsv') select 111 settings hdfs_truncate_on_insert=1;" + + +$CLICKHOUSE_CLIENT -q "select _size from hdfs('hdfs://localhost:12222/$CLICKHOUSE_TEST_UNIQUE_NAME.data*.tsv', auto, 'x UInt64') order by _size" +$CLICKHOUSE_CLIENT -q "select _size from hdfs('hdfs://localhost:12222/$CLICKHOUSE_TEST_UNIQUE_NAME.data*.tsv', auto, 'x UInt64') order by _size" +