Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ class AzureIteratorAsync final : public IObjectStorageIteratorAsync
}

private:
bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) override
bool getBatchAndCheckNext(PathsWithMetadata & batch) override
{
ProfileEvents::increment(ProfileEvents::AzureListObjects);
if (client->IsClientForDisk())
Expand All @@ -78,7 +78,7 @@ class AzureIteratorAsync final : public IObjectStorageIteratorAsync

for (const auto & blob : blobs_list)
{
batch.emplace_back(std::make_shared<RelativePathWithMetadata>(
batch.emplace_back(std::make_shared<PathWithMetadata>(
blob.Name,
ObjectMetadata{
static_cast<uint64_t>(blob.BlobSize),
Expand Down Expand Up @@ -160,7 +160,7 @@ ObjectStorageIteratorPtr AzureObjectStorage::iterate(const std::string & path_pr
return std::make_shared<AzureIteratorAsync>(path_prefix, client_ptr, max_keys ? max_keys : settings_ptr->list_object_keys_size);
}

void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const
void AzureObjectStorage::listObjects(const std::string & path, PathsWithMetadata & children, size_t max_keys) const
{
auto client_ptr = client.get();

Expand All @@ -182,7 +182,7 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith

for (const auto & blob : blobs_list)
{
children.emplace_back(std::make_shared<RelativePathWithMetadata>(
children.emplace_back(std::make_shared<PathWithMetadata>(
blob.Name,
ObjectMetadata{
static_cast<uint64_t>(blob.BlobSize),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ class AzureObjectStorage : public IObjectStorage

bool supportsListObjectsCache() override { return true; }

void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override;
void listObjects(const std::string & path, PathsWithMetadata & children, size_t max_keys) const override;

/// Sanitizer build may crash with max_keys=1; this looks like a false positive.
ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const override;
Expand Down
2 changes: 1 addition & 1 deletion src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ void CachedObjectStorage::copyObject( // NOLINT
object_storage->copyObject(object_from, object_to, read_settings, write_settings, object_to_attributes);
}

void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const
void CachedObjectStorage::listObjects(const std::string & path, PathsWithMetadata & children, size_t max_keys) const
{
object_storage->listObjects(path, children, max_keys);
}
Expand Down
2 changes: 1 addition & 1 deletion src/Disks/ObjectStorages/Cached/CachedObjectStorage.h
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ class CachedObjectStorage final : public IObjectStorage
IObjectStorage & object_storage_to,
std::optional<ObjectAttributes> object_to_attributes = {}) override;

void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override;
void listObjects(const std::string & path, PathsWithMetadata & children, size_t max_keys) const override;

ObjectMetadata getObjectMetadata(const std::string & path) const override;

Expand Down
4 changes: 2 additions & 2 deletions src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) co
return metadata;
}

void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const
void HDFSObjectStorage::listObjects(const std::string & path, PathsWithMetadata & children, size_t max_keys) const
{
initializeHDFSFS();
LOG_TEST(log, "Trying to list files for {}", path);
Expand Down Expand Up @@ -203,7 +203,7 @@ void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithM
}
else
{
children.emplace_back(std::make_shared<RelativePathWithMetadata>(
children.emplace_back(std::make_shared<PathWithMetadata>(
String(file_path),
ObjectMetadata{
static_cast<uint64_t>(ls.file_info[i].mSize),
Expand Down
2 changes: 1 addition & 1 deletion src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ class HDFSObjectStorage : public IObjectStorage, public HDFSErrorWrapper
const WriteSettings & write_settings,
std::optional<ObjectAttributes> object_to_attributes = {}) override;

void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override;
void listObjects(const std::string & path, PathsWithMetadata & children, size_t max_keys) const override;

String getObjectsNamespace() const override { return ""; }

Expand Down
12 changes: 6 additions & 6 deletions src/Disks/ObjectStorages/IObjectStorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -29,20 +29,20 @@ const MetadataStorageMetrics & IObjectStorage::getMetadataStorageMetrics() const

bool IObjectStorage::existsOrHasAnyChild(const std::string & path) const
{
RelativePathsWithMetadata files;
PathsWithMetadata files;
listObjects(path, files, 1);
return !files.empty();
}

void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata &, size_t) const
void IObjectStorage::listObjects(const std::string &, PathsWithMetadata &, size_t) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listObjects() is not supported");
}


ObjectStorageIteratorPtr IObjectStorage::iterate(const std::string & path_prefix, size_t max_keys) const
{
RelativePathsWithMetadata files;
PathsWithMetadata files;
listObjects(path_prefix, files, max_keys);

return std::make_shared<ObjectStorageIteratorFromList>(std::move(files));
Expand Down Expand Up @@ -101,7 +101,7 @@ WriteSettings IObjectStorage::patchSettings(const WriteSettings & write_settings
return write_settings;
}

void RelativePathWithMetadata::loadMetadata(ObjectStoragePtr object_storage, bool ignore_non_existent_file)
void PathWithMetadata::loadMetadata(ObjectStoragePtr object_storage, bool ignore_non_existent_file)
{
if (!metadata)
{
Expand All @@ -118,7 +118,7 @@ void RelativePathWithMetadata::loadMetadata(ObjectStoragePtr object_storage, boo
}
}

RelativePathWithMetadata::CommandInTaskResponse::CommandInTaskResponse(const std::string & task)
PathWithMetadata::CommandInTaskResponse::CommandInTaskResponse(const std::string & task)
{
Poco::JSON::Parser parser;
try
Expand All @@ -138,7 +138,7 @@ RelativePathWithMetadata::CommandInTaskResponse::CommandInTaskResponse(const std
}
}

std::string RelativePathWithMetadata::CommandInTaskResponse::to_string() const
std::string PathWithMetadata::CommandInTaskResponse::to_string() const
{
Poco::JSON::Object json;
if (retry_after_us.has_value())
Expand Down
27 changes: 20 additions & 7 deletions src/Disks/ObjectStorages/IObjectStorage.h
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ struct ObjectMetadata
ObjectAttributes attributes;
};

struct RelativePathWithMetadata
struct PathWithMetadata
{
class CommandInTaskResponse
{
Expand All @@ -124,23 +124,28 @@ struct RelativePathWithMetadata
String relative_path;
std::optional<ObjectMetadata> metadata;
CommandInTaskResponse command;
String absolute_path;

RelativePathWithMetadata() = default;
PathWithMetadata() = default;

explicit RelativePathWithMetadata(const String & task_string, std::optional<ObjectMetadata> metadata_ = std::nullopt)
explicit PathWithMetadata(const String & task_string, std::optional<ObjectMetadata> metadata_ = std::nullopt, String absolute_path_ = "")
: metadata(std::move(metadata_))
, command(task_string)
{
if (!command.is_parsed())
{
relative_path = task_string;
absolute_path = absolute_path_;
}
}

virtual ~RelativePathWithMetadata() = default;
virtual ~PathWithMetadata() = default;

virtual std::string getFileName() const { return std::filesystem::path(relative_path).filename(); }
virtual std::string getFileNameWithoutExtension() const { return std::filesystem::path(relative_path).stem(); }

virtual std::string getPath() const { return relative_path; }
virtual std::string getAbsolutePath() const { return absolute_path; }
virtual bool isArchive() const { return false; }
virtual std::string getPathToArchive() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not an archive"); }
virtual size_t fileSizeInArchive() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not an archive"); }
Expand All @@ -162,8 +167,8 @@ struct ObjectKeyWithMetadata
{}
};

using RelativePathWithMetadataPtr = std::shared_ptr<RelativePathWithMetadata>;
using RelativePathsWithMetadata = std::vector<RelativePathWithMetadataPtr>;
using PathWithMetadataPtr = std::shared_ptr<PathWithMetadata>;
using PathsWithMetadata = std::vector<PathWithMetadataPtr>;
using ObjectKeysWithMetadata = std::vector<ObjectKeyWithMetadata>;

class IObjectStorageIterator;
Expand Down Expand Up @@ -204,7 +209,7 @@ class IObjectStorage
virtual bool existsOrHasAnyChild(const std::string & path) const;

/// List objects recursively by certain prefix.
virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const;
virtual void listObjects(const std::string & path, PathsWithMetadata & children, size_t max_keys) const;

/// List objects recursively by certain prefix. Use it instead of listObjects, if you want to list objects lazily.
virtual ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const;
Expand Down Expand Up @@ -286,6 +291,14 @@ class IObjectStorage
/// buckets in S3. If object storage doesn't have any namepaces return empty string.
virtual String getObjectsNamespace() const = 0;

virtual std::unique_ptr<IObjectStorage> cloneObjectStorage(
const std::string &,
const Poco::Util::AbstractConfiguration &,
const std::string &, ContextPtr)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'cloneObjectStorage' is not implemented");
}

/// Generate blob name for passed absolute local path.
/// Path can be generated either independently or based on `path`.
virtual ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const = 0;
Expand Down
4 changes: 2 additions & 2 deletions src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ ObjectMetadata LocalObjectStorage::getObjectMetadata(const std::string & path) c
return object_metadata;
}

void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t/* max_keys */) const
void LocalObjectStorage::listObjects(const std::string & path, PathsWithMetadata & children, size_t/* max_keys */) const
{
if (!fs::exists(path) || !fs::is_directory(path))
return;
Expand All @@ -164,7 +164,7 @@ void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWith
continue;
}

children.emplace_back(std::make_shared<RelativePathWithMetadata>(entry.path(), getObjectMetadata(entry.path())));
children.emplace_back(std::make_shared<PathWithMetadata>(entry.path(), getObjectMetadata(entry.path())));
}
}

Expand Down
2 changes: 1 addition & 1 deletion src/Disks/ObjectStorages/Local/LocalObjectStorage.h
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ class LocalObjectStorage : public IObjectStorage

ObjectMetadata getObjectMetadata(const std::string & path) const override;

void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override;
void listObjects(const std::string & path, PathsWithMetadata & children, size_t max_keys) const override;

bool existsOrHasAnyChild(const std::string & path) const override;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ std::vector<std::string> MetadataStorageFromPlainObjectStorage::listDirectory(co
{
auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize();

RelativePathsWithMetadata files;
PathsWithMetadata files;
std::string absolute_key = key_prefix;
if (!absolute_key.ends_with('/'))
absolute_key += '/';
Expand Down
2 changes: 1 addition & 1 deletion src/Disks/ObjectStorages/ObjectStorageIterator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}

RelativePathWithMetadataPtr ObjectStorageIteratorFromList::current()
PathWithMetadataPtr ObjectStorageIteratorFromList::current()
{
if (!isValid())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator");
Expand Down
18 changes: 9 additions & 9 deletions src/Disks/ObjectStorages/ObjectStorageIterator.h
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,10 @@ class IObjectStorageIterator
virtual bool isValid() = 0;

/// Return the current element.
virtual RelativePathWithMetadataPtr current() = 0;
virtual PathWithMetadataPtr current() = 0;

/// This will initiate prefetching the next batch in background, so it can be obtained faster when needed.
virtual std::optional<RelativePathsWithMetadata> getCurrentBatchAndScheduleNext() = 0;
virtual std::optional<PathsWithMetadata> getCurrentBatchAndScheduleNext() = 0;

/// Returns the number of elements in the batches that were fetched so far.
virtual size_t getAccumulatedSize() const = 0;
Expand All @@ -36,7 +36,7 @@ class IObjectStorageIterator
/// Return the current batch of elements.
/// It is unspecified how batches are formed.
/// But this method can be used for more efficient processing.
virtual RelativePathsWithMetadata currentBatch() = 0;
virtual PathsWithMetadata currentBatch() = 0;
};

using ObjectStorageIteratorPtr = std::shared_ptr<IObjectStorageIterator>;
Expand All @@ -45,7 +45,7 @@ class ObjectStorageIteratorFromList : public IObjectStorageIterator
{
public:
/// Everything is represented by just a single batch.
explicit ObjectStorageIteratorFromList(RelativePathsWithMetadata && batch_)
explicit ObjectStorageIteratorFromList(PathsWithMetadata && batch_)
: batch(std::move(batch_))
, batch_iterator(batch.begin()) {}

Expand All @@ -59,11 +59,11 @@ class ObjectStorageIteratorFromList : public IObjectStorageIterator

bool isValid() override { return batch_iterator != batch.end(); }

RelativePathWithMetadataPtr current() override;
PathWithMetadataPtr current() override;

RelativePathsWithMetadata currentBatch() override { return batch; }
PathsWithMetadata currentBatch() override { return batch; }

std::optional<RelativePathsWithMetadata> getCurrentBatchAndScheduleNext() override
std::optional<PathsWithMetadata> getCurrentBatchAndScheduleNext() override
{
if (batch.empty())
return {};
Expand All @@ -76,8 +76,8 @@ class ObjectStorageIteratorFromList : public IObjectStorageIterator
size_t getAccumulatedSize() const override { return batch.size(); }

private:
RelativePathsWithMetadata batch;
RelativePathsWithMetadata::iterator batch_iterator;
PathsWithMetadata batch;
PathsWithMetadata::iterator batch_iterator;
};

}
6 changes: 3 additions & 3 deletions src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ bool IObjectStorageIteratorAsync::isValid()
return !is_finished;
}

RelativePathWithMetadataPtr IObjectStorageIteratorAsync::current()
PathWithMetadataPtr IObjectStorageIteratorAsync::current()
{
std::lock_guard lock(mutex);

Expand All @@ -123,7 +123,7 @@ RelativePathWithMetadataPtr IObjectStorageIteratorAsync::current()
}


RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch()
PathsWithMetadata IObjectStorageIteratorAsync::currentBatch()
{
std::lock_guard lock(mutex);

Expand All @@ -133,7 +133,7 @@ RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch()
return current_batch;
}

std::optional<RelativePathsWithMetadata> IObjectStorageIteratorAsync::getCurrentBatchAndScheduleNext()
std::optional<PathsWithMetadata> IObjectStorageIteratorAsync::getCurrentBatchAndScheduleNext()
{
std::lock_guard lock(mutex);

Expand Down
14 changes: 7 additions & 7 deletions src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h
Original file line number Diff line number Diff line change
Expand Up @@ -23,24 +23,24 @@ class IObjectStorageIteratorAsync : public IObjectStorageIterator

bool isValid() override;

RelativePathWithMetadataPtr current() override;
RelativePathsWithMetadata currentBatch() override;
PathWithMetadataPtr current() override;
PathsWithMetadata currentBatch() override;

void next() override;
void nextBatch() override;

size_t getAccumulatedSize() const override;
std::optional<RelativePathsWithMetadata> getCurrentBatchAndScheduleNext() override;
std::optional<PathsWithMetadata> getCurrentBatchAndScheduleNext() override;

void deactivate();

protected:
/// This method fetches the next batch, and returns true if there are more batches after it.
virtual bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) = 0;
virtual bool getBatchAndCheckNext(PathsWithMetadata & batch) = 0;

struct BatchAndHasNext
{
RelativePathsWithMetadata batch;
PathsWithMetadata batch;
bool has_next;
};

Expand All @@ -55,8 +55,8 @@ class IObjectStorageIteratorAsync : public IObjectStorageIterator
ThreadPool list_objects_pool;
ThreadPoolCallbackRunnerUnsafe<BatchAndHasNext> list_objects_scheduler;
std::future<BatchAndHasNext> outcome_future;
RelativePathsWithMetadata current_batch;
RelativePathsWithMetadata::iterator current_batch_iterator;
PathsWithMetadata current_batch;
PathsWithMetadata::iterator current_batch_iterator;
std::atomic<size_t> accumulated_size = 0;
};

Expand Down
Loading
Loading