DerGut commented on PR #1368:
URL: https://github.com/apache/iceberg-rust/pull/1368#issuecomment-2931598528

   🗞️ 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]](https://apache.github.io/hadoop/hadoop-aws/tools/hadoop-aws/index.html), 
[[2]](https://apache.github.io/hadoop/hadoop-azure/index.html)) 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](https://github.com/fsspec/adlfs/blob/adb9c53b74a0d420625b86dd00fbe615b43201d2/adlfs/spec.py#L236)
 expects.
   
   Of course, supporting both could also be an option.
   
   ### 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](https://github.com/apache/iceberg/blob/a4b2a0dab092821d4843749b8abc30208622e164/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java#L66-L69)
 ships support for the `wasb[s]://` scheme using the same FileIO 
implementation. Supporting it in pyiceberg [is in 
~discussion](https://github.com/apache/iceberg-python/pull/1663).
   
   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 
(https://github.com/Azure/Azurite/issues/553). In fact, I've tried using it 
with the OpenDAL `services-azdls` and didn't get it to work.
   <details>
   
   <summary>Test setup</summary>
   
   ```sh
   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;";
   ```
   ```rs
   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`
   ```
   </details>
   
   On the other hand, both 
[pyiceberg](https://github.com/apache/iceberg-python/blob/91853898c89f5376aa7fe874e8c2c6eb2ca224b2/dev/docker-compose-azurite.yml#L17)
 and [Iceberg 
Java](https://github.com/apache/iceberg/blob/a4b2a0dab092821d4843749b8abc30208622e164/azure/src/integration/java/org/apache/iceberg/azure/adlsv2/AzuriteContainer.java#L32)
 seem to be able to use Azurite for their test setup. Partially I've understood 
that pyiceberg's ADLS FileIO implementation 
([[1]](https://github.com/apache/iceberg-python/blob/91853898c89f5376aa7fe874e8c2c6eb2ca224b2/pyiceberg/io/fsspec.py#L194),
 
[[2]](https://github.com/fsspec/adlfs/blob/adb9c53b74a0d420625b86dd00fbe615b43201d2/adlfs/spec.py#L35))
 is built on Blob Storage directly instead of ADLSv2. At the same time, 
[Iceberg 
Java](https://github.com/apache/iceberg/blob/a4b2a0dab092821d4843749b8abc30208622e164/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java#L23)
 seems to use Azure's ADLS client instead. I need to dive deep
 er 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://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](https://github.com/apache/iceberg-rust/pull/1368#discussion_r2102303084).
 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 😬 
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to