mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge pull request #65386 from skyoct/feat-s3-field
Feat add _etag for object storage
This commit is contained in:
commit
315fd5496a
@ -146,6 +146,7 @@ Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Reading fr
|
||||
- `_file` — Name of the file. Type: `LowCardinalty(String)`.
|
||||
- `_size` — Size of the file in bytes. Type: `Nullable(UInt64)`. If the size is unknown, the value is `NULL`.
|
||||
- `_time` — Last modified time of the file. Type: `Nullable(DateTime)`. If the time is unknown, the value is `NULL`.
|
||||
- `_etag` — ETag of the file. Type: `LowCardinalty(String)`. If the etag is unknown, the value is `NULL`.
|
||||
|
||||
For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns).
|
||||
|
||||
|
@ -86,6 +86,7 @@ private:
|
||||
Poco::Timestamp::fromEpochTime(
|
||||
std::chrono::duration_cast<std::chrono::seconds>(
|
||||
static_cast<std::chrono::system_clock::time_point>(blob.Details.LastModified).time_since_epoch()).count()),
|
||||
blob.Details.ETag.ToString(),
|
||||
{}}));
|
||||
}
|
||||
|
||||
@ -186,6 +187,7 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith
|
||||
Poco::Timestamp::fromEpochTime(
|
||||
std::chrono::duration_cast<std::chrono::seconds>(
|
||||
static_cast<std::chrono::system_clock::time_point>(blob.Details.LastModified).time_since_epoch()).count()),
|
||||
blob.Details.ETag.ToString(),
|
||||
{}}));
|
||||
}
|
||||
|
||||
|
@ -205,7 +205,7 @@ void DiskObjectStorageMetadata::addObject(ObjectStorageKey key, size_t size)
|
||||
}
|
||||
|
||||
total_size += size;
|
||||
keys_with_meta.emplace_back(std::move(key), ObjectMetadata{size, {}, {}});
|
||||
keys_with_meta.emplace_back(std::move(key), ObjectMetadata{size, {}, {}, {}});
|
||||
}
|
||||
|
||||
ObjectKeyWithMetadata DiskObjectStorageMetadata::popLastObject()
|
||||
|
@ -222,6 +222,7 @@ void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithM
|
||||
ObjectMetadata{
|
||||
static_cast<uint64_t>(ls.file_info[i].mSize),
|
||||
Poco::Timestamp::fromEpochTime(ls.file_info[i].mLastMod),
|
||||
"",
|
||||
{}}));
|
||||
}
|
||||
|
||||
|
@ -54,6 +54,7 @@ struct ObjectMetadata
|
||||
{
|
||||
uint64_t size_bytes = 0;
|
||||
Poco::Timestamp last_modified;
|
||||
std::string etag;
|
||||
ObjectAttributes attributes;
|
||||
};
|
||||
|
||||
|
@ -146,7 +146,7 @@ private:
|
||||
auto objects = outcome.GetResult().GetContents();
|
||||
for (const auto & object : objects)
|
||||
{
|
||||
ObjectMetadata metadata{static_cast<uint64_t>(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}};
|
||||
ObjectMetadata metadata{static_cast<uint64_t>(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), object.GetETag(), {}};
|
||||
batch.emplace_back(std::make_shared<RelativePathWithMetadata>(object.GetKey(), std::move(metadata)));
|
||||
}
|
||||
|
||||
@ -332,6 +332,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet
|
||||
ObjectMetadata{
|
||||
static_cast<uint64_t>(object.GetSize()),
|
||||
Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()),
|
||||
object.GetETag(),
|
||||
{}}));
|
||||
|
||||
if (max_keys)
|
||||
@ -479,6 +480,7 @@ ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) cons
|
||||
ObjectMetadata result;
|
||||
result.size_bytes = object_info.size;
|
||||
result.last_modified = Poco::Timestamp::fromEpochTime(object_info.last_modification_time);
|
||||
result.etag = object_info.etag;
|
||||
result.attributes = object_info.metadata;
|
||||
|
||||
return result;
|
||||
|
@ -54,6 +54,7 @@ namespace
|
||||
ObjectInfo object_info;
|
||||
object_info.size = static_cast<size_t>(result.GetContentLength());
|
||||
object_info.last_modification_time = result.GetLastModified().Seconds();
|
||||
object_info.etag = result.GetETag();
|
||||
|
||||
if (with_metadata)
|
||||
object_info.metadata = result.GetMetadata();
|
||||
|
@ -15,6 +15,7 @@ struct ObjectInfo
|
||||
{
|
||||
size_t size = 0;
|
||||
time_t last_modification_time = 0;
|
||||
String etag;
|
||||
|
||||
std::map<String, String> metadata = {}; /// Set only if getObjectInfo() is called with `with_metadata = true`.
|
||||
};
|
||||
|
@ -204,7 +204,9 @@ Chunk StorageObjectStorageSource::generate()
|
||||
{.path = getUniqueStoragePathIdentifier(*configuration, *object_info, false),
|
||||
.size = object_info->isArchive() ? object_info->fileSizeInArchive() : object_info->metadata->size_bytes,
|
||||
.filename = &filename,
|
||||
.last_modified = object_info->metadata->last_modified});
|
||||
.last_modified = object_info->metadata->last_modified,
|
||||
.etag = &(object_info->metadata->etag)
|
||||
});
|
||||
|
||||
const auto & partition_columns = configuration->getPartitionColumns();
|
||||
if (!partition_columns.empty() && chunk_size && chunk.hasColumns())
|
||||
|
@ -116,7 +116,7 @@ void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & blo
|
||||
|
||||
NameSet getVirtualNamesForFileLikeStorage()
|
||||
{
|
||||
return {"_path", "_file", "_size", "_time"};
|
||||
return {"_path", "_file", "_size", "_time", "_etag"};
|
||||
}
|
||||
|
||||
VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription & storage_columns)
|
||||
@ -135,6 +135,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(const ColumnsDescription
|
||||
add_virtual("_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()));
|
||||
add_virtual("_size", makeNullable(std::make_shared<DataTypeUInt64>()));
|
||||
add_virtual("_time", makeNullable(std::make_shared<DataTypeDateTime>()));
|
||||
add_virtual("_etag", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()));
|
||||
|
||||
return desc;
|
||||
}
|
||||
@ -230,6 +231,13 @@ void addRequestedFileLikeStorageVirtualsToChunk(
|
||||
else
|
||||
chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst());
|
||||
}
|
||||
else if (virtual_column.name == "_etag")
|
||||
{
|
||||
if (virtual_values.etag)
|
||||
chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), (*virtual_values.etag))->convertToFullColumnIfConst());
|
||||
else
|
||||
chunk.addColumn(virtual_column.type->createColumnConstWithDefaultValue(chunk.getNumRows())->convertToFullColumnIfConst());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -83,7 +83,7 @@ struct VirtualsForFileLikeStorage
|
||||
std::optional<size_t> size { std::nullopt };
|
||||
const String * filename { nullptr };
|
||||
std::optional<Poco::Timestamp> last_modified { std::nullopt };
|
||||
|
||||
const String * etag { nullptr };
|
||||
};
|
||||
|
||||
void addRequestedFileLikeStorageVirtualsToChunk(
|
||||
|
@ -11,5 +11,5 @@ create table test_02245_2 (a UInt64, _path Int32) engine = S3(s3_conn, filename=
|
||||
insert into test_02245_2 select 1, 2 settings s3_truncate_on_insert=1;
|
||||
select * from test_02245_2;
|
||||
1 2
|
||||
select _path from test_02245_2;
|
||||
2
|
||||
select _path, isNotNull(_etag) from test_02245_2;
|
||||
2 1
|
||||
|
@ -12,4 +12,4 @@ drop table if exists test_02245_2;
|
||||
create table test_02245_2 (a UInt64, _path Int32) engine = S3(s3_conn, filename='test_02245_2', format=Parquet);
|
||||
insert into test_02245_2 select 1, 2 settings s3_truncate_on_insert=1;
|
||||
select * from test_02245_2;
|
||||
select _path from test_02245_2;
|
||||
select _path, isNotNull(_etag) from test_02245_2;
|
||||
|
@ -247,6 +247,7 @@ DoubleDelta
|
||||
Doxygen
|
||||
Durre
|
||||
ECMA
|
||||
ETag
|
||||
Ecto
|
||||
EdgeAngle
|
||||
EdgeLengthKm
|
||||
@ -1587,6 +1588,7 @@ enum's
|
||||
enums
|
||||
erfc
|
||||
errorCodeToName
|
||||
etag
|
||||
evalMLMethod
|
||||
exFAT
|
||||
expiryMsec
|
||||
|
Loading…
Reference in New Issue
Block a user