Skip to content

feat(storage-azdls): Add Azure Datalake Storage support #1368

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 8 commits into from
Jun 5, 2025

Conversation

DerGut
Copy link
Contributor

@DerGut DerGut commented May 22, 2025

Which issue does this PR close?

What changes are included in this PR?

This PR adds an integration for the Azure Datalake storage service. At it's core, it adds parsing logic for configuration properties. The finished config struct is simply passed down to OpenDAL. In addition it adds logic to parse fully qualified file URIs, and matches it against expected (previously configured) values.

It also creates a new Storage::Azdls enum variant based on OpenDAL's existing Scheme::Azdls enum variant. It then fits the parsing logic into the existing framework to build the storage integration from an io::FileIOBuilder.

Note on WASB support

Other Iceberg ADLS integrations (pyiceberg + Java) also support the wasb:// and wasbs:// schemes.
WASB refers to a client-side implementation of hierarchical namespaces on top of Blob Storage. ADLS(v2) on the other hand is a service offered by Azure, also built on top of Blob Storage.
IIUC we can accept both schemes because objects written to Blob Storage via wasb:// will also be accessible via adfs:// (which operates on the same Blob Storage).
Even though the URIs slightly differ in format when they refer to the same object, we can largely reuse existing logic.

-wasb[s]://<containername>@<accountname>.blob.core.windows.net/<path>
+adfs[s]://<filesystemname>@<accountname>.dfs.core.windows.net/<path>

Are these changes tested?

Unit

I added minor unit tests to validate the configuration property parsing logic.

Integration

I decided not to add integration tests because

  1. ADLS is not S3-compatible which means that we can't reuse our Minio setup
  2. the Azure-specific alternative to local testing - Azurite - doesn't support ADLS

End-to-end

I have yet to test it in a functioning environment.

DerGut added 2 commits May 22, 2025 12:17
Signed-off-by: Jannik Steinmann <jannik.steinmann@datadoghq.com>
Signed-off-by: Jannik Steinmann <jannik.steinmann@datadoghq.com>
@DerGut DerGut changed the title Add ADLS storage support feat(storage-azdls): Add Azure Datalake Storage support May 22, 2025
let mut cfg = AzdlsConfig::default();

if let Some(_conn_str) = m.remove(ADLS_CONNECTION_STRING) {
return Err(Error::new(
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When we get connection string parsing into OpenDAL, we should be able to call something like AzdlsConfig::try_from_connection_string() here instead.

Then, we can also mark the ADLS_CONNECTION_STRING constant as public.

Copy link
Contributor Author

@DerGut DerGut Jun 2, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently, the endpoint is inferred from the fully specified paths passed to FileIO. Other implementations like pyiceberg and Iceberg Java additionally accept a connection string property, but Java for example doesn't allow the user to set the endpoint explicitly.

I'd suggest to release without connection string parsing because it's already usable, but not add an endpoint property for now to not clutter the configuration options.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did some reordering here. I felt like it was more obvious to separate the feature-flagged mod and use statements from the non-feature-flagged ones. The #[cfg(... lines made it easy for me to overlook that something non-integration related was declared (e.g. pub(crate) mod object_cache;).

I also separated the feature-flagged mod and use statements. This allowed my IDE able to sort them, which I think makes sense with a growing number of storage integrations.

@Xuanwo
Copy link
Member

Xuanwo commented May 23, 2025

Nice work, @DerGut! Most of the changes in this PR look good to me. We can merge it after the next opendal release, which will include most of what we need here.

The other option is to add AZDLS support first without client_secret settings, and then include them in following PRs.

Signed-off-by: Jannik Steinmann <jannik.steinmann@datadoghq.com>
DerGut added 5 commits June 2, 2025 00:46
Signed-off-by: Jannik Steinmann <jannik.steinmann@datadoghq.com>
Signed-off-by: Jannik Steinmann <jannik.steinmann@datadoghq.com>
The filesystem will always be empty by the point we construct the operator, so there's not point in validating it.

Signed-off-by: Jannik Steinmann <jannik.steinmann@datadoghq.com>
Signed-off-by: Jannik Steinmann <jannik.steinmann@datadoghq.com>
@DerGut
Copy link
Contributor Author

DerGut commented Jun 2, 2025

🗞️ Since this was already taking longer than expected, here is an update about what I learned in the past week or so 🎉

Some of these were misconceptions that led me to revamp parts of the PR, others are simply learnings that might be worth sharing/ documenting.

Path Format

I wasn't sure which path format to expect and ended up revamping the PR to use fully qualified path notation (e.g. abfss://<myfs>@<myaccount>.dfs.core.windows.net/mydir/myfile.parquet). From what I've found ([1], [2]) it seemed the most widely used format to specify Azure Storage objects. We also use it internally. Other options are a shortened format (e.g. abfss://mydir/myfile.parquet). This would be more consistent with other storage path forms (like s3:// or gs://) and seems to be what pyiceberg expects.

Of course, supporting both could also be an option.

Update: While S3 and GCS enforce globally unique bucket names, Azure Storage only enforces uniqueness per URI. This means that next to the container/ filesystem name, we also need to carry the account name and endpoint suffix around. While this can technically be dropped assuming an operator has been configured with this information, I think it is more consistent with other uses of Azure Storage URIs to pass along this information.

WASB

ADLSv2, Azure's most recent HDFS-compatible storage service uses the abfs[s]:// protocol scheme. It's built on Blob Storage and comes with HDFS abstractions as a service. IIUC there was a before (even before ADLSv1), when WASB implemented similar APIs directly on Blob Storage, but client-side.
Iceberg Java ships support for the wasb[s]:// scheme using the same FileIO implementation. Supporting it in pyiceberg is in ~discussion.

It took me a while to wrap my head around this. But in essence I understand:

  1. wasb can access the same objects because they are ultimately stored in some Blob Storage container
  2. we can treat wasb://*.blob.* paths as abfs://*.dfs* paths and simply use the server-side ADLS-v2 APIs

Azurite

I don't yet fully understand how Azurite (Azure's local dev storage) fits into all of this. On the one hand, it seems like it only supports the Azure Blob Storage API, and ADLSv2 APIs are missing (Azure/Azurite#553). In fact, I've tried using it with the OpenDAL services-azdls and didn't get it to work.

Test setup
docker run -d -p 10000:10000 mcr.microsoft.com/azure-storage/azurite
az storage fs create --name test --connection-string "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;"
let builder = Azdls::default()
    .filesystem("test") // Created this one above
    .endpoint("http://127.0.0.1:10000/devstoreaccount1")
    .account_name("devstoreaccount1")
   .account_key("Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==");

let op = Operator::new(builder).unwrap().finish();
op.list("/").await.unwrap();  // Fails, but succeeds for the equivalent `Azblob`

On the other hand, both pyiceberg and Iceberg Java seem to be able to use Azurite for their test setup. Partially I've understood that pyiceberg's ADLS FileIO implementation ([1], [2]) is built on Blob Storage directly instead of ADLSv2. At the same time, Iceberg Java seems to use Azure's ADLS client instead. I need to dive deeper to understand why they are able to use Azurite in the Java implementation.

Endpoints

Azure Storage uses different endpoints for different services. For example, Blob Storage uses the https://<account>.blob.<suffix> endpoint while ADLS uses the https://<account>.dfs.<suffix> one. These are the endpoints the underlying HTTP client will use to send requests to.
The current PR implementation expects a fully-qualified path to objects/ files in ADLS. This means we can construct the endpoint from any path, e.g. abfss://myfs@myaccount.dfs.core.windows.net/dir/file.parquet would become the endpoint https://myaccount.dfs.core.windows.net.

In Azure SDKs, endpoints can either be set by an explicit endpoint configuration option, or by passing a connection string. The current PR implementation will validate that a configured endpoint will match what's defined in a fully qualified path.

If we decide to roll with the fully qualified path format, I'd suggest to keep the configuration options as they are now because users aren't required to configure the endpoint explicitly.
If we decide to use the short path notation instead, we could either introduce a new endpoint property (only pyiceberg has it, Java relies on the connection string), or wait for a new OpenDAL version to introduce connection string support.


Also to reply to your earlier comment

The other option is to add AZDLS support first without client_secret settings, and then include them in following PRs.

Since I was taking so long, this is now already included 😬

@DerGut
Copy link
Contributor Author

DerGut commented Jun 2, 2025

🏁 My plan to get this PR to the finish line

  • abandon Azurite support (for now): Unfortunately this sacrifices integration tests and easy local development. But for the latter at least, users can still iterate with a different FileIO implementation
  • test it: I'm planning some file operations against a real ADLS account

🙏 And what I need from the community/ reviewers

  • Path format: Do we want to settle on the long form, the short one or both? -> long
  • Defensive/ typed vs. concise/ dynamic-ish code: I've defaulted to a more defensive programming approach when implementing the path+configuration parsing+matching logic. This also helped me to understand what I was working with. To be fair, this bloats the code and if preferred we can still simplify it.

@DerGut
Copy link
Contributor Author

DerGut commented Jun 5, 2025

✅ I tested that the FileIO operations are working, and I created an example adjacent to the OSS one to validate the catalog can use it too.

I also double-checked whether the LocationGenerator is able to work with the fully qualified paths. But since it only operates on the path part of the metadata, I don't think it justifies a unit test.

@DerGut DerGut marked this pull request as ready for review June 5, 2025 12:54
Copy link
Member

@Xuanwo Xuanwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks really nice, thank you for working on this!

@Xuanwo
Copy link
Member

Xuanwo commented Jun 5, 2025

One question I have is whether we really need to keep the original abfss scheme. I don't recall any situation where we need to use it, perhaps only when creating a table? I wonder if we could simply remove it after parsing the input path.

@Xuanwo Xuanwo merged commit b5b8aa8 into apache:main Jun 5, 2025
18 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

FileIO storage support for ADLSv2
2 participants