Skip to content
Merged
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
25 changes: 24 additions & 1 deletion src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h
Original file line number Diff line number Diff line change
Expand Up @@ -278,6 +278,7 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration,
std::string getEngineName() const override { return getImpl().getEngineName(); }
std::string getNamespaceType() const override { return getImpl().getNamespaceType(); }

Path getFullPath() const override { return getImpl().getFullPath(); }
Path getPath() const override { return getImpl().getPath(); }
void setPath(const Path & path) override { getImpl().setPath(path); }

Expand All @@ -294,9 +295,14 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration,
ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure) override
{ getImpl().addStructureAndFormatToArgsIfNeeded(args, structure_, format_, context, with_structure); }

bool withPartitionWildcard() const override { return getImpl().withPartitionWildcard(); }
bool withGlobsIgnorePartitionWildcard() const override { return getImpl().withGlobsIgnorePartitionWildcard(); }
bool isPathWithGlobs() const override { return getImpl().isPathWithGlobs(); }
bool isNamespaceWithGlobs() const override { return getImpl().isNamespaceWithGlobs(); }
std::string getPathWithoutGlobs() const override { return getImpl().getPathWithoutGlobs(); }

bool isArchive() const override { return getImpl().isArchive(); }
bool isPathInArchiveWithGlobs() const override { return getImpl().isPathInArchiveWithGlobs(); }
std::string getPathInArchive() const override { return getImpl().getPathInArchive(); }

void check(ContextPtr context) const override { getImpl().check(context); }
Expand Down Expand Up @@ -338,8 +344,19 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration,
std::optional<ColumnsDescription> tryGetTableStructureFromMetadata() const override
{ return getImpl().tryGetTableStructureFromMetadata(); }

bool supportsFileIterator() const override { return getImpl().supportsFileIterator(); }
ObjectIterator iterate(
const ActionsDAG * filter_dag,
std::function<void(FileProgress)> callback,
size_t list_batch_size) override
{
return getImpl().iterate(filter_dag, callback, list_batch_size);
}

void update(ObjectStoragePtr object_storage, ContextPtr local_context) override
{ return getImpl().update(object_storage, local_context); }
void updateIfRequired(ObjectStoragePtr object_storage, ContextPtr local_context) override
{ return getImpl().updateIfRequired(object_storage, local_context); }

void initialize(
ASTs & engine_args,
Expand All @@ -356,7 +373,11 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration,
return getImpl().createArgsWithAccessData();
}

protected:
const StorageObjectStorageSettings & getSettingsRef() const override
{
return getImpl().getSettingsRef();
}

void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override
{ return getImpl().fromNamedCollection(collection, context); }
void fromAST(ASTs & args, ContextPtr context, bool with_structure) override
Expand Down Expand Up @@ -439,6 +460,8 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration,
createDynamicStorage(type);
}

virtual void assertInitialized() const override { return getImpl().assertInitialized(); }

private:
inline StorageObjectStorage::Configuration & getImpl() const
{
Expand Down
16 changes: 8 additions & 8 deletions src/Storages/ObjectStorage/StorageObjectStorage.h
Original file line number Diff line number Diff line change
Expand Up @@ -204,15 +204,15 @@ class StorageObjectStorage::Configuration
virtual void addStructureAndFormatToArgsIfNeeded(
ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure) = 0;

bool withPartitionWildcard() const;
virtual bool withPartitionWildcard() const;
bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); }
bool withGlobsIgnorePartitionWildcard() const;
bool isPathWithGlobs() const;
bool isNamespaceWithGlobs() const;
virtual bool withGlobsIgnorePartitionWildcard() const;
virtual bool isPathWithGlobs() const;
virtual bool isNamespaceWithGlobs() const;
virtual std::string getPathWithoutGlobs() const;

virtual bool isArchive() const { return false; }
bool isPathInArchiveWithGlobs() const;
virtual bool isPathInArchiveWithGlobs() const;
virtual std::string getPathInArchive() const;

virtual void check(ContextPtr context) const;
Expand Down Expand Up @@ -261,23 +261,23 @@ class StorageObjectStorage::Configuration
String structure = "auto";

virtual void update(ObjectStoragePtr object_storage, ContextPtr local_context);
void updateIfRequired(ObjectStoragePtr object_storage, ContextPtr local_context);
virtual void updateIfRequired(ObjectStoragePtr object_storage, ContextPtr local_context);

/// Create arguments for table function with path and access parameters
virtual ASTPtr createArgsWithAccessData() const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method createArgsWithAccessData is not supported by storage {}", getEngineName());
}

const StorageObjectStorageSettings & getSettingsRef() const;
virtual const StorageObjectStorageSettings & getSettingsRef() const;

virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0;
virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0;

virtual ObjectStorageType extractDynamicStorageType(ASTs & /* args */, ContextPtr /* context */, ASTPtr * /* type_arg */ = nullptr) const
{ return ObjectStorageType::None; }

void assertInitialized() const;
virtual void assertInitialized() const;

bool initialized = false;
std::atomic<bool> updated = false;
Expand Down
2 changes: 1 addition & 1 deletion src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -283,7 +283,7 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded(
{"s3", "s3Cluster"},
{"azureBlobStorage", "azureBlobStorageCluster"},
{"hdfs", "hdfsCluster"},
{"iceberg", "icebergS3Cluster"},
{"iceberg", "icebergCluster"},
{"icebergS3", "icebergS3Cluster"},
{"icebergAzure", "icebergAzureCluster"},
{"icebergHDFS", "icebergHDFSCluster"},
Expand Down
16 changes: 16 additions & 0 deletions src/Storages/ObjectStorage/StorageObjectStorageCluster.h
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,22 @@ class StorageObjectStorageCluster : public IStorageCluster

void addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const override;

std::optional<UInt64> totalRows(ContextPtr query_context) const override
{
if (pure_storage)
return pure_storage->totalRows(query_context);
configuration->update(object_storage, query_context);
return configuration->totalRows();
}

std::optional<UInt64> totalBytes(ContextPtr query_context) const override
{
if (pure_storage)
return pure_storage->totalBytes(query_context);
configuration->update(object_storage, query_context);
return configuration->totalBytes();
}

private:
void updateQueryToSendIfNeeded(
ASTPtr & query,
Expand Down
Loading
Loading