Skip to content
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

Enable reading WASB and WASBS file paths with ABFS and ABFSS #10127

Closed
njriasan opened this issue Apr 12, 2024 · 7 comments · Fixed by #11294 or #11504 · May be fixed by #11830
Closed

Enable reading WASB and WASBS file paths with ABFS and ABFSS #10127

njriasan opened this issue Apr 12, 2024 · 7 comments · Fixed by #11294 or #11504 · May be fixed by #11830
Labels
improvement PR that improves existing functionality SNOWFLAKE

Comments

@njriasan
Copy link

Feature Request / Improvement

When you setup a managed Snowflake Iceberg table on an Azure account, they will provide locations that use wasbs:// and not abfss://. wasb is currently deprecated by Azure and everyone is encouraged to use ABFS instead. While Snowflake should really change this behavior, in the spirit of allowing people to "update" Iceberg tables without rewriting all the metadata files, it would be great if the iceberg library could handle this automatically.

Its my understanding that you can convert a wasbs URI to an abfss URI by just making two changes:

  • Replacing wasbs:// with abfss://
  • Updating blob.core.windows.net with dfs.core.windows.net.

If this change could be replaced when loading the location and file information from metadata files, then every user could effortlessly update to using abfs.

Query engine

Snowflake

@njriasan njriasan added the improvement PR that improves existing functionality label Apr 12, 2024
@njriasan
Copy link
Author

I marked this as Snowflake because this is the offending writer, but I think this generally would apply to all engines.

@njriasan
Copy link
Author

I'm also interesting in making this contribution to Iceberg if others are okay with this change and can give me some pointers about the best way to ensure this can be applied to every file path.

@ms1111
Copy link

ms1111 commented Apr 19, 2024

Is there a difference in that wasbs is for blob storage, and abfss requires hierarchical storage (ADLSv2) to be enabled? If you try to read a blob storage account using the ADLS SDK, certain things will not work.

(iceberg-azure seems to require abfss & ADLSv2. I was actually wishing it'd work with wasbs, using the blob storage APIs, because ADLSv2 doesn't support automatic blob versioning.)

@dennishuo
Copy link
Contributor

@ms1111 raises a good point, as there are some known incompatibilities in low-level Blob vs ADLS APIs: https://learn.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-known-issues

However, it looks like the discrepancies may or may not be fundamental to Iceberg/Hadoop use cases depending on what subset of each APIs the different client-side implementations use.

It looks like while the legacy Hadoop wasbs impl is being deprecated, some of the underlying Blob APIs are still being used both by the legacy BlobClient as well as newer DataLake*Client. For example, the DataLakeFileSystemClientBuilder appears to always simultaneously create both an internal "datalake client" as well as a "blob client", pointing respectively at dfs.core.windows.net and blob.core.windows.net: https://github.com/Azure/azure-sdk-for-java/blob/0aa45226a625aa19da7183800bb90531eb1f1ee2/sdk/storage/azure-storage-file-datalake/src/main/java/com/azure/storage/file/datalake/DataLakeFileSystemClientBuilder.java#L175

public DataLakeFileSystemClientBuilder endpoint(String endpoint) {
    // Ensure endpoint provided is dfs endpoint
    endpoint = DataLakeImplUtils.endpointToDesiredEndpoint(endpoint, "dfs", "blob");
    blobContainerClientBuilder.endpoint(DataLakeImplUtils.endpointToDesiredEndpoint(endpoint, "blob", "dfs"));

And the helper function doesn't really care if the input is dfs or blob already, it just constructs with the desired one: https://github.com/Azure/azure-sdk-for-java/blob/21eb8bc8b4cce3e365bbcb9d139b07a3a554a2d9/sdk/storage/azure-storage-file-datalake/src/main/java/com/azure/storage/file/datalake/implementation/util/DataLakeImplUtils.java#L16

public static String endpointToDesiredEndpoint(String endpoint, String desiredEndpoint, String currentEndpoint) {
    // Add the . on either side to prevent overwriting an account name.
    String desiredStringToMatch = "." + desiredEndpoint + ".";
    String currentRegexToMatch = "\\." + currentEndpoint + "\\.";
    if (endpoint.contains(desiredStringToMatch)) {
        return endpoint;
    } else {
        return endpoint.replaceFirst(currentRegexToMatch, desiredStringToMatch);
    }
}

And then some of the methods just delegate through to the "blob client" instead of the "datalake client": https://github.com/Azure/azure-sdk-for-java/blob/0aa45226a625aa19da7183800bb90531eb1f1ee2/sdk/storage/azure-storage-file-datalake/src/main/java/com/azure/storage/file/datalake/DataLakeFileClient.java#L1079

public FileReadResponse readWithResponse(OutputStream stream, FileRange range, DownloadRetryOptions options,
    DataLakeRequestConditions requestConditions, boolean getRangeContentMd5, Duration timeout, Context context) {
    return DataLakeImplUtils.returnOrConvertException(() -> {
        BlobDownloadResponse response = blockBlobClient.downloadWithResponse(stream, Transforms.toBlobRange(range),
            Transforms.toBlobDownloadRetryOptions(options), Transforms.toBlobRequestConditions(requestConditions),
            getRangeContentMd5, timeout, context);
        return Transforms.toFileReadResponse(response);
    }, LOGGER);
}

While it'll be important for all service providers to also migrate to no longer producing wasbs:// paths by default and ensuring users all have ADLSv2 enabled, it seems to me the Iceberg metadata/manifest-list/manifest files will be fairly sticky for lots of large Iceberg deployments with exabytes of data involved, so even if new data is migrated quickly, the libraries really need to support ingesting wasbs:// scheme for the foreseeable future.

Are there any Azure experts who can confirm that ADLSFileIO strictly adheres to the subset of DataLakeFileSystemClient that only depends on the Blob semantics? If so, could that mean it's possible to use DataLakeFileSystemClient even for blob storage accounts that don't enable ADLSv2?

If it'll be guaranteed to remain drop-in compatible, it seems like one approach could be to include wasbs:// in ResolvingFileIO to also map to ADLSFileIO and then make ADLSLocation more permissive to accept wasbs:// URIs. Looks like it may not technically even need to do any explicit path-translation to abfss since the scheme prefix abfss is discarded anyways, and the code I pasted above from Azure client libraries seems to not actually care too much about whether you're trying to configure the client with dfs.core.windows.net or blob.core.windows.net.

@dennishuo
Copy link
Contributor

A cursory check through https://github.com/apache/iceberg/blob/main/azure/src/main/java/org/apache/iceberg/azure/adlsv2 seems to only reveal three methods used:

And all three appear to just delegate through to the internal blockBlobClient instead of touching dataLakeFileAsyncClient, so it does seem like simply accepting wasbs in ADLSFileIO should be sufficient. Would be good to have a second set of eyes confirm though.

It could also be safest to just make ADLSLocation permissive in accepting wasbs:// without changing ResolvingFileIO if people want wasbs:// to still automatically fallthrough to HadoopFileIO in usual scenarios.

@njriasan I'm happy to help review a PR if you wanted to take a stab at this (we'll naturally still need a committer to also review).

@RussellSpitzer
Copy link
Member

Bringing this back up, it seems like this is a general Iceberg Issue. I think we should just add WASB and WASBS to the resolving FileIO as suggested above.

@liko9
Copy link
Contributor

liko9 commented Nov 1, 2024

Should we reopen this issue since it was reverted and still exists as a problem?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
improvement PR that improves existing functionality SNOWFLAKE
Projects
None yet
6 participants