apache / incubator-xtable

Apache XTable (incubating) is a cross-table converter for lakehouse table formats that facilitates interoperability across data processing systems and query engines.
https://xtable.apache.org/
Apache License 2.0
930 stars 149 forks source link

Unable to generate Delta metadata for Snowflake-created Iceberg #287

Open bkvarda opened 11 months ago

bkvarda commented 11 months ago

Created Iceberg data in S3 following this tutorial

Using Shallow Clone in Databricks with Photon enabled we can generate Delta metadata from the created Iceberg in S3. However when attempting to do the same with OneTable, we receive the following error:

2023-12-05 23:48:19 INFO  org.apache.spark.sql.delta.InitialSnapshot:57 - [tableId=62953b04-8f92-428f-b2bf-cbaa4daf2554] Created snapshot InitialSnapshot(path=s3://brandon-kvarda-sandbox-data/iceberg/customer/_delta_log, version=-1, metadata=Metadata(30358631-745f-4dc5-b638-47570a0e2219,null,null,Format(parquet,Map()),null,List(),Map(),Some(1701820099184)), logSegment=LogSegment(s3://brandon-kvarda-sandbox-data/iceberg/customer/_delta_log,-1,List(),List(),None,-1), checksumOpt=None)
2023-12-05 23:48:19 INFO  io.onetable.client.OneTableClient:236 - No previous OneTable sync for target. Falling back to snapshot sync.
2023-12-05 23:48:19 WARN  org.apache.iceberg.hadoop.HadoopTableOperations:325 - Error reading version hint file s3://brandon-kvarda-sandbox-data/iceberg/customer/metadata/version-hint.text
java.lang.NumberFormatException: For input string: "1701803553322000000"
    at ormatException.forInputString(NumberFormatException.java:65) ~[?:?]
    at java.lang.Integer.parseInt(Integer.java:652) ~[?:?]
    at java.lang.Integer.parseInt(Integer.java:770) ~[?:?]
    at org.apache.iceberg.hadoop.HadoopTableOperations.findVersion(HadoopTableOperations.java:320) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]java.lang.NumberF
the-other-tim-brown commented 11 months ago

Is 1701803553322000000 the value in your version hint file? That is a lot of commits.

If I'm reading the link correctly, you need to use the Snowflake Iceberg catalog org.apache.iceberg.snowflake.SnowflakeCatalog - did you configure this in your sync?

the-other-tim-brown commented 11 months ago

@sagarlakshmipathy since I think you are also curious in trying out this flow.

bkvarda commented 11 months ago

did you configure this in your sync?

@the-other-tim-brown I did try to use that in my config actually but I couldn't get it to pick up that class even when I shaded it into the onetable jar. But it's definitely possible I didn't have the config done correctly - what should it look like?

Is 1701803553322000000 the value in your version hint file? That is a lot of commits.

Yeah it is, and that is indeed what is in the version hint file. I'm not exactly sure how Snowflake comes up with that value, but it actually looks like it is an epoch (seems to line up with roughly when I ran this test).

sagarlakshmipathy commented 11 months ago

You're right @bkvarda , I have noticed that snowflake writes the metadata file with the epoch ts.

For the actual issue: Can you provide me with minimal steps to reproduce this error?

bkvarda commented 11 months ago

@sagarlakshmipathy all the steps are in the blog I mentioned - I followed that to get to the point where I had iceberg data created by snowflake in s3. The only additional things I did were register the same aws role created for the Snowflake volume as an instance profile in Databricks, built the OneTable jar and tried to run it on a basic config to turn iceberg into delta (pointing at the s3 path).

sagarlakshmipathy commented 11 months ago

@bkvarda thanks for the explanation. I was able to replicate this issue. Breaking this down further, the issue itself has nothing to with Databricks. Its more of OneTable not liking the naming convention that Snowflake uses.

For my notes: minimal steps to reproduce this error:

  1. Create external volume and grant access to the S3 bucket : https://docs.snowflake.com/user-guide/tables-iceberg-configure-external-volume#configure-an-external-volume-for-amazon-s3
    
    CREATE OR REPLACE EXTERNAL VOLUME onetable_complete_dataset
    STORAGE_LOCATIONS =
      (
         (
            NAME = 'my-s3-us-west-2'
            STORAGE_PROVIDER = 'S3'
            STORAGE_BASE_URL = 's3://budket-name/'
            STORAGE_AWS_ROLE_ARN = 'arn:aws:iam::acct-id:role/SnowflakeS3AccessRole'
         )
      );

DESC EXTERNAL VOLUME onetable_complete_dataset;

GRANT ALL ON EXTERNAL VOLUME onetable_complete_dataset TO ROLE iceberg_lab WITH GRANT OPTION;


3. Create ICEBERG table in Snowflake

CREATE OR REPLACE ICEBERG TABLE people (id NUMBER(38, 0), name VARCHAR, age NUMBER(38, 0), city VARCHAR, create_ts VARCHAR) CATALOG='SNOWFLAKE' EXTERNAL_VOLUME='onetable_complete_dataset' BASE_LOCATION='snowflake_created/people';

INSERT INTO people (id, name, age, city, create_ts) VALUES (1, 'John', 25, 'NYC', '2023-09-28 00:00:00'), (2, 'Emily', 30, 'SFO', '2023-09-28 00:00:00'), (3, 'Michael', 35, 'ORD', '2023-09-28 00:00:00'), (4, 'Andrew', 40, 'NYC', '2023-10-28 00:00:00'), (5, 'Bob', 28, 'SEA', '2023-09-23 00:00:00'), (6, 'Charlie', 31, 'DFW', '2023-08-29 00:00:00');

SELECT * FROM people;


Bundle the OneTable jar and run it against:

sourceFormat: ICEBERG targetFormats:

WARNING: Runtime environment or build system does not support multi-release JARs. This will impact location-based features.
2023-12-20 18:46:25 INFO  io.onetable.utilities.RunSync:150 - Running sync for basePath s3://onetable-testing-vs/snowflake_created/people/ for following table formats [HUDI, DELTA]
2023-12-20 18:46:25 INFO  org.apache.hudi.common.table.HoodieTableMetaClient:133 - Loading HoodieTableMetaClient from s3://onetable-testing-vs/snowflake_created/people/data
2023-12-20 18:46:26 WARN  org.apache.hadoop.util.NativeCodeLoader:60 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
2023-12-20 18:46:26 WARN  org.apache.hadoop.metrics2.impl.MetricsConfig:136 - Cannot locate configuration: tried hadoop-metrics2-s3a-file-system.properties,hadoop-metrics2.properties
2023-12-20 18:46:27 INFO  io.onetable.hudi.HudiTableManager:73 - Hudi table does not exist, will be created on first sync
WARNING: An illegal reflective access operation has occurred
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/sagarl/oss/onetable/utilities/target/utilities-0.1.0-SNAPSHOT-bundled.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Please consider reporting this to the maintainers of org.apache.spark.unsafe.Platform
WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations
WARNING: All illegal access operations will be denied in a future release
2023-12-20 18:46:29 INFO  org.apache.spark.sql.delta.storage.DelegatingLogStore:57 - LogStore `LogStoreAdapter(io.delta.storage.S3SingleDriverLogStore)` is used for scheme `s3`
2023-12-20 18:46:30 INFO  org.apache.spark.sql.delta.DeltaLog:57 - Creating initial snapshot without metadata, because the directory is empty
2023-12-20 18:46:30 INFO  org.apache.spark.sql.delta.InitialSnapshot:57 - [tableId=258a0111-a6ba-4d27-a2d9-d001b10cf956] Created snapshot InitialSnapshot(path=s3://onetable-testing-vs/snowflake_created/people/data/_delta_log, version=-1, metadata=Metadata(37631afb-18a7-4513-bb98-10c1ac0e08fd,null,null,Format(parquet,Map()),null,List(),Map(),Some(1703126790155)), logSegment=LogSegment(s3://onetable-testing-vs/snowflake_created/people/data/_delta_log,-1,List(),List(),None,-1), checksumOpt=None)
2023-12-20 18:46:30 INFO  io.onetable.client.OneTableClient:236 - No previous OneTable sync for target. Falling back to snapshot sync.
2023-12-20 18:46:30 INFO  io.onetable.client.OneTableClient:236 - No previous OneTable sync for target. Falling back to snapshot sync.
2023-12-20 18:46:30 WARN  org.apache.iceberg.hadoop.HadoopTableOperations:325 - Error reading version hint file s3://onetable-testing-vs/snowflake_created/people/metadata/version-hint.text
java.lang.NumberFormatException: For input string: "1703126411625000000"
        at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) ~[?:?]
        at java.lang.Integer.parseInt(Integer.java:652) ~[?:?]
        at java.lang.Integer.parseInt(Integer.java:770) ~[?:?]
        at org.apache.iceberg.hadoop.HadoopTableOperations.findVersion(HadoopTableOperations.java:320) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at org.apache.iceberg.hadoop.HadoopTableOperations.refresh(HadoopTableOperations.java:104) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at org.apache.iceberg.hadoop.HadoopTableOperations.current(HadoopTableOperations.java:84) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at org.apache.iceberg.hadoop.HadoopTables.load(HadoopTables.java:94) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.iceberg.IcebergTableManager.lambda$getTable$1(IcebergTableManager.java:58) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at java.util.Optional.orElseGet(Optional.java:369) [?:?]
        at io.onetable.iceberg.IcebergTableManager.getTable(IcebergTableManager.java:58) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.iceberg.IcebergSourceClient.initSourceTable(IcebergSourceClient.java:82) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.iceberg.IcebergSourceClient.getSourceTable(IcebergSourceClient.java:61) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.iceberg.IcebergSourceClient.getCurrentSnapshot(IcebergSourceClient.java:134) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.spi.extractor.ExtractFromSource.extractSnapshot(ExtractFromSource.java:35) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.client.OneTableClient.syncSnapshot(OneTableClient.java:179) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.client.OneTableClient.sync(OneTableClient.java:117) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.utilities.RunSync.main(RunSync.java:171) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
2023-12-20 18:46:31 INFO  org.apache.iceberg.hadoop.HadoopTables:101 - Table location loaded: s3://onetable-testing-vs/snowflake_created/people
2023-12-20 18:46:31 ERROR io.onetable.utilities.RunSync:173 - Error running sync for s3://onetable-testing-vs/snowflake_created/people/
java.lang.NullPointerException: null
        at io.onetable.iceberg.IcebergSourceClient.getTable(IcebergSourceClient.java:95) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.iceberg.IcebergSourceClient.getCurrentSnapshot(IcebergSourceClient.java:137) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.spi.extractor.ExtractFromSource.extractSnapshot(ExtractFromSource.java:35) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.client.OneTableClient.syncSnapshot(OneTableClient.java:179) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.client.OneTableClient.sync(OneTableClient.java:117) ~[utilities-0.1.0-SNAPSHOT-bundled.jar:?]
        at io.onetable.utilities.RunSync.main(RunSync.java:171) [utilities-0.1.0-SNAPSHOT-bundled.jar:?]
(base) sagarl@dev onetable % 

@the-other-tim-brown : this will impact all SF written tables

sagarlakshmipathy commented 10 months ago

As we are looking into more robust options to get this working. One workaround (super hacky) that I tested to get this to work is as follows:

  1. Copy the v1703126411625000000.metadata.json file and paste it in the same folder with a different name i.e. v2.metadata.json
  2. Update the version-hint.text in metadata/ folder and add 2 (because the new file name we named is v2.metadata.json) at the last new line
  3. Run sync with below config
sourceFormat: ICEBERG
targetFormats:
  - HUDI
  - DELTA
datasets:
  -
    tableBasePath: s3://bucket-name/snowflake_created/people/
    tableDataPath: s3://bucket-name/snowflake_created/people/data/
    tableName: people

In the longer term we should look into reading the metadata from the SF catalog itself.

the-other-tim-brown commented 10 months ago

@bkvarda you can try something similar to what another user did with BigLake Metastore https://github.com/onetable-io/onetable/issues/313#issuecomment-1905331896

They were able to add the required jar to the path and read the source information from the catalog directly.

sagarlakshmipathy commented 10 months ago

@bkvarda @the-other-tim-brown

These configs worked for me.

config.yaml

sourceFormat: ICEBERG
targetFormats:
  - DELTA
datasets:
  -
    tableBasePath: s3://<bucket-name>/snowflake_created/test_people
    tableDataPath: s3://<bucket-name>/snowflake_created/test_people/data
    tableName: test_people
    namespace: onetable_testing.public

catalog.yaml

catalogImpl: org.apache.iceberg.snowflake.SnowflakeCatalog
catalogName: snowflake_catalog
catalogOptions:
  io-impl: org.apache.iceberg.aws.s3.S3FileIO
  warehouse: s3://<bucket-name>/warehouse
  uri: jdbc:snowflake://<account-identifier>.snowflakecomputing.com
  jdbc.user: <snowflake-username>
  jdbc.password: <snowflake-password>

java -cp /Users/sagarl/Downloads/iceberg-spark-runtime-3.2_2.12-1.4.2.jar:utilities/target/utilities-0.1.0-SNAPSHOT-bundled.jar:/Users/sagarl/Downloads/snowflake-jdbc-3.13.28.jar:/Users/sagarl/Downloads/iceberg-aws-1.4.2.jar://Users/sagarl/Downloads/bundle-2.23.9.jar io.onetable.utilities.RunSync --datasetConfig config.yaml --icebergCatalogConfig catalog.yaml

Output:

2024-01-25 16:35:09 INFO  io.onetable.client.OneTableClient:128 - Sync is successful for the following formats DELTA

Ref: https://docs.snowflake.com/en/user-guide/tables-iceberg-catalog

Note: I had some issue reading the table as Delta using spark. i.e. : org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 8.0 failed 1 times, most recent failure: Lost task 0.0 in stage 8.0 (TID 102) (10.0.0.199 executor driver): java.lang.UnsupportedOperationException: Unsupported encoding: DELTA_BINARY_PACKED

I'll look into this.

Update:

I was able to read the table through Amazon Athena without any issues.