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

Wrong conversion from Delta to Iceberg when Delta table is enabled column mapping #624

Open
2 of 4 tasks
emilie-wang opened this issue Jan 17, 2025 · 5 comments
Open
2 of 4 tasks
Labels
bug Something isn't working

Comments

@emilie-wang
Copy link

Search before asking

  • I had searched in the issues and found no similar issues.

Please describe the bug 🐞

Error

When converting a Delta table to Iceberg table with enabling Delta column mapping(https://docs.delta.io/latest/delta-column-mapping.html), I got this error.

java.lang.IllegalArgumentException: Multiple entries with same key: 1=id and 1=info.element
	at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.conflictException(ImmutableMap.java:378) ~[iceberg-bundled-guava-1.4.2.jar:?]
	at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.checkNoConflict(ImmutableMap.java:372) ~[iceberg-bundled-guava-1.4.2.jar:?]
...
	at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap$Builder.build(ImmutableMap.java:587) ~[iceberg-bundled-guava-1.4.2.jar:?]
	at org.apache.iceberg.types.IndexByName.byId(IndexByName.java:81) ~[iceberg-api-1.4.2.jar:?]
	at org.apache.iceberg.types.TypeUtil.indexNameById(TypeUtil.java:172) ~[iceberg-api-1.4.2.jar:?]
	at org.apache.iceberg.Schema.lazyIdToName(Schema.java:183) ~[iceberg-api-1.4.2.jar:?]
	at org.apache.iceberg.Schema.<init>(Schema.java:112) ~[iceberg-api-1.4.2.jar:?]
	at org.apache.iceberg.Schema.<init>(Schema.java:91) ~[iceberg-api-1.4.2.jar:?]
	at org.apache.iceberg.Schema.<init>(Schema.java:83) ~[iceberg-api-1.4.2.jar:?]
	at org.apache.iceberg.Schema.<init>(Schema.java:79) ~[iceberg-api-1.4.2.jar:?]
	at org.apache.xtable.iceberg.IcebergSchemaExtractor.toIceberg(IcebergSchemaExtractor.java:79) ~[classes/:?]
	at org.apache.xtable.iceberg.IcebergConversionTarget.initializeTableIfRequired(IcebergConversionTarget.java:155) ~[classes/:?]
	at org.apache.xtable.iceberg.IcebergConversionTarget.beginSync(IcebergConversionTarget.java:143) ~[classes/:?]
	at org.apache.xtable.spi.sync.TableFormatSync.getSyncResult(TableFormatSync.java:154) ~[classes/:?]
	at org.apache.xtable.spi.sync.TableFormatSync.syncSnapshot(TableFormatSync.java:70) [classes/:?]
	at org.apache.xtable.conversion.ConversionController.syncSnapshot(ConversionController.java:182) [classes/:?]
	at org.apache.xtable.conversion.ConversionController.sync(ConversionController.java:118) [classes/:?]
	at org.apache.xtable.utilities.RunSync.main(RunSync.java:191) [classes/:?]
2025-01-13 14:45:35 ERROR org.apache.xtable.conversion.ConversionController:135 - Sync failed for the following formats ICEBERG

Details with troubleshooting

Table schema:

scala> df.printSchema
root
 |-- id: integer (nullable = true)
 |-- name: string (nullable = true)
 |-- info: array (nullable = true)
 |    |-- element: string (containsNull = true)

Troubleshooting:

While enabling delta column mapping, Delta doesn't assign id for element level for a list type field (similar for map field), here is what looks like in Delta (log from _delta_log/00000000000000000000.json)
Raw log snippets
raw log

"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{\"delta.columnMapping.id\":1,\"delta.columnMapping.physicalName\":\"id\"}},{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"delta.columnMapping.id\":2,\"delta.columnMapping.physicalName\":\"name\"}},{\"name\":\"info\",\"type\":{\"type\":\"array\",\"elementType\":\"string\",\"containsNull\":true},\"nullable\":true,\"metadata\":{\"delta.columnMapping.id\":3,\"delta.columnMapping.physicalName\":\"info\"}}]}"

Readable structure:

{
    "type": "struct",
    "fields": [
        {
            "name": "id",
            "type": "integer",
            "nullable": true,
            "metadata": {
                "delta.columnMapping.id": 1,
                "delta.columnMapping.physicalName": "id"
            }
        },
        {
            "name": "name",
            "type": "string",
            "nullable": true,
            "metadata": {
                "delta.columnMapping.id": 2,
                "delta.columnMapping.physicalName": "name"
            }
        },
        {
            "name": "info",
            "type": {
                "type": "array",
                "elementType": "string",
                "containsNull": true
            },
            "nullable": true,
            "metadata": {
                "delta.columnMapping.id": 3,
                "delta.columnMapping.physicalName": "info"
            }
        }
    ]
}

However, Iceberg assigns id for element level when the field is list type. See how Xtable assigns ids for fields whose id is null without checking :

https://github.com/apache/incubator-xtable/blob/main/xtable-core/src/main/java/org/apache/xtable/iceberg/IcebergSchemaExtractor.java#L67
https://github.com/apache/incubator-xtable/blob/main/xtable-core/src/main/java/org/apache/xtable/iceberg/IcebergSchemaExtractor.java#L244

Are you willing to submit PR?

  • I am willing to submit a PR!
  • I am willing to submit a PR but need help getting started!

Code of Conduct

@emilie-wang emilie-wang added the bug Something isn't working label Jan 17, 2025
@emilie-wang
Copy link
Author

Hi @the-other-tim-brown, could you please take a look on this issue? Thank you.

@the-other-tim-brown
Copy link
Contributor

Thanks for reporting this, it seems like we need a concept of a "last used ID" instead of starting at 0, is that correct? I am curious if Iceberg will allow us to set the ID of the element level higher than the value's ID.

@emilie-wang
Copy link
Author

emilie-wang commented Jan 23, 2025

Hi Tim @the-other-tim-brown ,
Thank you for checking it. I believe it is one possible solution, but it is quite hard to assign not used ID to Iceberg, considering the schema evolution in Delta and new columns can be added in future. And further more I am also worried what you mentioned "if Iceberg will allow us to set the ID of the element level higher than the value's ID" and if Iceberg can load the parquet file correctly? So far I do not have a clear idea about the fix.
Since it is recommended by Xtable to enable the column mapping while do Delta to Iceberg conversion: https://xtable.apache.org/docs/features-and-limitations#delta and I was wondering if you can plan to priority the fix for this issue?

@emilie-wang
Copy link
Author

A quick update,, to answer this " if Iceberg will allow us to set the ID of the element level higher than the value's ID.", I tested by using same Schema without enabling column mappings. This is the schema of the converted Iceberg table, the ids are set from Xtable, while array field info's id is 3 and element-id is 4, which has no issue.

    "type" : "struct",
    "schema-id" : 0,
    "fields" : [ {
      "id" : 1,
      "name" : "id",
      "required" : false,
      "type" : "int"
    }, {
      "id" : 2,
      "name" : "name",
      "required" : false,
      "type" : "string"
    }, {
      "id" : 3,
      "name" : "info",
      "required" : false,
      "type" : {
        "type" : "list",
        "element-id" : 4,
        "element" : "string",
        "element-required" : false
      }
    } ]
  } ],

However, back to original issue for a Delta table with enabling column mapping, when the schema evolves and for example a new column is added, actually the column name written in the underlying parquet file is different:

        {
            "name": "type",
            "type": "string",
            "nullable": true,
            "metadata": {
                "delta.columnMapping.id": 4,
                "delta.columnMapping.physicalName": "col-1ee2c1d5-0785-4787-be84-a44a114cfa7d"
            }
        },

The added field name is type, but the physicalName in parquet file is col-1ee2c1d5-0785-4787-be84-a44a114cfa7d. If the field name type is carried to Iceberg schema, Iceberg can't read successfully from the parquet file.

@emilie-wang
Copy link
Author

Apart from it, I was wondering if you have tested the case when renaming a column of delta table, can Iceberg table successfully handle this schema update?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

No branches or pull requests

2 participants