Skip to content

[Bug]: ManagedIO for Iceberg throws NoSuchElementException when write is attempted after a delete #37199

@avtar-roambee

Description

@avtar-roambee

What happened?

While using ManagedIO (Managed.ICEBERG) for an Iceberg Table, I get the following exception when I attempt to insert (append) records after a prior delete.

Caused by: java.util.NoSuchElementException
	at org.apache.iceberg.relocated.com.google.common.collect.AbstractIndexedListIterator.next(AbstractIndexedListIterator.java:79)
	at org.apache.beam.sdk.io.iceberg.AppendFilesToTables$AppendFilesToTablesDoFn.shouldSkip(AppendFilesToTables.java:235)
	at org.apache.beam.sdk.io.iceberg.AppendFilesToTables$AppendFilesToTablesDoFn.processElement(AppendFilesToTables.java:133)

Looking at AppendFilesToTables.java Line 235

table.currentSnapshot().addedDataFiles(table.io()).iterator().next().path().toString();
It looks like the code is expecting that there will be some added files for sure in the current snapshot. However, if the latest snapshot has no added files but only delete files due to some delete operation, this returns an empty iterator causing the above exception.

Is this an expected behavior?

Issue Priority

Priority: 2 (default / most bugs should be filed as P2)

Issue Components

  • Component: Python SDK
  • Component: Java SDK
  • Component: Go SDK
  • Component: Typescript SDK
  • Component: IO connector
  • Component: Beam YAML
  • Component: Beam examples
  • Component: Beam playground
  • Component: Beam katas
  • Component: Website
  • Component: Infrastructure
  • Component: Spark Runner
  • Component: Flink Runner
  • Component: Samza Runner
  • Component: Twister2 Runner
  • Component: Hazelcast Jet Runner
  • Component: Google Cloud Dataflow Runner

Metadata

Metadata

Assignees

No one assigned

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions