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

[BUG][Kernel] Delta kernel read checkpoint parquet file failed with exception "list not found in optional group readerFeatures (LIST)" #3082

Closed
1 of 8 tasks
Youngwb opened this issue May 10, 2024 · 1 comment · Fixed by #3083
Labels
bug Something isn't working

Comments

@Youngwb
Copy link

Youngwb commented May 10, 2024

Bug

Which Delta project/connector is this regarding?

  • Spark
  • Standalone
  • Flink
  • Kernel
  • Other (fill in here)

Describe the problem

When use delta kernel to read delta lake table encountered such an error

list not found in optional group readerFeatures (LIST) {
  repeated group bag {
    optional binary array (STRING);
  }
}

and the schema of parquet file is
image

And I think delta kernel currently does not supports reading this legacy array type.

Steps to reproduce

Observed results

Expected results

Further details

Environment information

  • Delta Lake version:
  • Spark version:
  • Scala version:

Willingness to contribute

The Delta Lake Community encourages bug fix contributions. Would you or another member of your organization be willing to contribute a fix for this bug to the Delta Lake code base?

  • Yes. I can contribute a fix for this bug independently.
  • Yes. I would be willing to contribute a fix for this bug with guidance from the Delta Lake community.
  • No. I cannot contribute a bug fix at this time.
@Youngwb Youngwb added the bug Something isn't working label May 10, 2024
@vkorukanti
Copy link
Collaborator

@Youngwb Thanks for reporting. Currently, the default Parquet reader is not handling the legacy list physical format. Will post a PR shortly.

vkorukanti added a commit that referenced this issue May 10, 2024
…repeated types (#3083)

## Description
When legacy mode is enabled in Spark, array physical types are stored
slightly different from the standard format.

Standard mode (default):
```
optional group readerFeatures (LIST) {
  repeated group list {
    optional binary element (STRING);
  }
}
```

When write legacy mode is enabled (`spark.sql.parquet.writeLegacyFormat
= true`):
```
optional group readerFeatures (LIST) {
  repeated group bag {
    optional binary array (STRING);
  }
}
```

TODO: We need to handle the 2-level lists. Will post a separate PR. The
challenge is with generating or finding the Parquet files with 2-level
lists.

## How was this patch tested?
Added tests

Fixes #3082
vkorukanti added a commit that referenced this issue May 10, 2024
…repeated types (#3083)

## Description
When legacy mode is enabled in Spark, array physical types are stored
slightly different from the standard format.

Standard mode (default):
```
optional group readerFeatures (LIST) {
  repeated group list {
    optional binary element (STRING);
  }
}
```

When write legacy mode is enabled (`spark.sql.parquet.writeLegacyFormat
= true`):
```
optional group readerFeatures (LIST) {
  repeated group bag {
    optional binary array (STRING);
  }
}
```

TODO: We need to handle the 2-level lists. Will post a separate PR. The
challenge is with generating or finding the Parquet files with 2-level
lists.

## How was this patch tested?
Added tests

Fixes #3082
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

Successfully merging a pull request may close this issue.

2 participants