Merge pull request #57126 from Avogar/size-virtual-column

Add _size virtual column to s3/file/hdfs/url/azureBlobStorage engines
This commit is contained in:
Kruglov Pavel 2023-11-27 10:12:18 +01:00 committed by GitHub
commit b10e46b2bc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
33 changed files with 217 additions and 43 deletions

View File

@ -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)

View File

@ -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}

View File

@ -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).

View File

@ -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.

View File

@ -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.

View File

@ -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)

View File

@ -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.

View File

@ -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}

View File

@ -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.

View File

@ -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}

View File

@ -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<size_t> 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;
}

View File

@ -181,6 +181,7 @@ private:
std::unique_ptr<QueryPipeline> pipeline;
std::unique_ptr<PullingPipelineExecutor> reader;
String current_path;
std::optional<size_t> current_file_size;
/// Recreate ReadBuffer and PullingPipelineExecutor for each file.
bool initialize();

View File

@ -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)

View File

@ -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;
}
}

View File

@ -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(); });

View File

@ -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;
}

View File

@ -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)

View File

@ -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<size_t> current_file_size;
struct stat current_archive_stat;
std::optional<String> filename_override;
Block sample_block;

View File

@ -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<StorageS3Source::IIterator> createFileIterator(

View File

@ -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<size_t> getFileSize() const { return key_with_info->info ? std::optional(key_with_info->info->size) : std::nullopt; }
const IInputFormat * getInputFormat() const { return dynamic_cast<const IInputFormat *>(source.get()); }

View File

@ -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)

View File

@ -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<size_t> 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;
}

View File

@ -225,6 +225,7 @@ private:
Block block_for_format;
std::shared_ptr<IteratorWrapper> uri_iterator;
Poco::URI curr_uri;
std::optional<size_t> current_file_size;
String format;
const std::optional<FormatSettings> & format_settings;
HTTPHeaderEntries headers;

View File

@ -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)

View File

@ -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<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{"_size", makeNullable(std::make_shared<DataTypeUInt64>())}};
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)
{
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<DataTypeUInt64>(), "_idx"});
@ -409,7 +413,10 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const
{
Block block;
for (const auto & column : virtual_columns)
{
if (column.name == "_file" || column.name == "_path")
block.insert({column.type->createColumn(), column.type, column.name});
}
block.insert({ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "_idx"});
for (size_t i = 0; i != paths.size(); ++i)
@ -420,8 +427,8 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & 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_t> 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());
}
}
}

View File

@ -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<T> & sources, const std::vector<String> & 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_t> size, const String * filename = nullptr);
}
}

View File

@ -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"
)

View File

@ -0,0 +1,12 @@
2
3
4
2
3
4
2
3
4
2
3
4

View File

@ -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.*

View File

@ -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

View File

@ -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"

View File

@ -0,0 +1,6 @@
2
3
4
2
3
4

View File

@ -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"