Iceberg Table Corruption and Data Loss in the Wild: Part 1

You’re a data engineer, and for a moment - everything looks calm. Dashboards are green, ingestion is steady, lags are low. Then, a worrying Slack message from a data scientist: “Do you know what this error means?”

In this post, we want to share a story about a sneaky bug we encountered that caused table corruption, as well as silent data loss in Iceberg tables. If you're using Iceberg, if your ingestion is based on a streaming pipeline, if you're an AWS EMR user, or if you just like a good bug hunt - read on.
Background
Apache Iceberg is a specification for defining tables on Object Storage. It defines a set of metadata files which together describe a table. things such as its schema, how it's partitioned, and most importantly: where its data resides.
Iceberg table data is stored in data files, which are typically Parquet files stored in Object Storage, such as S3.
Engines (such as Spark, Flink, Trino and more) interact with Iceberg tables. They read and write metadata files to locate and manipulate data files when executing user queries.
Users and applications access Iceberg tables through these engines. When issues occur with either data files or metadata files, it's the engine that reports the failure.
The Problem
Two different exceptions began appearing simultaneously. We suspected they were connected rather than just coincidental unrelated errors (as you'll see later - this assumption was right).
Different queries triggered one or more of these exceptions:
java.io.IOException: java.lang.RuntimeException: s3://xxxxx.parquet is not a Parquet file. expected magic number at tail [80, 65, 82, 49] but found [x, x, x, x]
Parquet is the most prominent format for data files in Iceberg (perhaps in data lakes generally). Parquet files are marked by a magic "PAR1" signature at both the beginning and end of the file.
The first exception indicated that this expected magic signature wasn't found where it should have been, causing the read to fail.
software.amazon.awssdk.services.s3.model.S3Exception: The requested range is not satisfiable
The second exception occurred when the reader requested an invalid portion of the S3 file.
Interestingly, these exceptions weren't thrown at the same location in the code. We'll soon discover why.
Data Loss and Table Corruption
Ryft monitors Iceberg tables in real time to make smart decisions about what can be optimized and when. This monitoring revealed something concerning. Data files were being written to the same path multiple times with different sizes and record counts.
In other words, this was a data corruption issue. Moreover, it was also a silent data loss issue. Files were overwritten during normal ingestion, without any explicit indication. When an exception was eventually thrown, it was when some of your data was already long gone.
Thanks to our continuous monitoring, we could definitively confirm that files were being rewritten, and we could identify exactly when it happened and which files were affected. This would be impossible to determine by examining only the current state of the table.
Reading Iceberg Code
Files were being overwritten, and it was time to find out why. We compared the tables with this corruption issue to other tables which were fine. We noticed something interesting in the keys of the data files of the different tables:
Here is an example key of an overwritten file:
00011-12345-afafafaf-1010-1111-ddee-1e2e3e4e5e6e-00001.parquet
And here is an example of a key from a healthy table:
00011-56321-efde3e12-1010-2356-efde-1e2e3e4e5e6e-274171-00001.parquet
Note the part in bold. it looks like an extra portion of the generated ID, which is only present in keys from the healthy table. What's more, we saw files in the healthy table that had the exact same path except for the part in bold.
To understand exactly how object keys are generated, we examined the source code. Searching the apache/iceberg repository, we quickly found the following pull request:

This confirmed our suspicion: it was a known issue. Because the problem did not cause errors during writes, and manifested as various different exceptions, it was particularly difficult to identify through online searches.
The fix was introduced in Iceberg version 1.5.0.
Explaining the Fix
Before the pull request, file names were generated by using IDs that are unique per Spark Application alongside a running, in-memory counter**.** In batch mode, this isn't a problem. application IDs are unique, and in case of failure, the next run will have a different one.
However, in Spark Streaming, application IDs persist even across restarts. When a streaming application restarted, the ID remained the same while the counter reset, causing name collisions and file overwrites.
The pull request fixed the issue by adding the epoch id, which changes between restarts, to the name of generated files.
Bonus Section: Explaining Exceptions
This section goes deeper into the explanation for each of the three exceptions we encountered. It includes some elaboration on Iceberg metadata and how query engines use it to scan efficiently.
If you're not interested in these technical details and just want to see how to protect yourself, feel free to skip directly to the conclusions section.
At the beginning of this post, we mentioned 2 different exceptions:
java.io.IOException: java.lang.RuntimeException: hdfs://xxxxx.parquet is not a Parquet file. expected magic number at tail [80, 65, 82, 49] but found [x, x, x, x]
software.amazon.awssdk.services.s3.model.S3Exception: The requested range is not satisfiable
Knowing that files are being overwritten, all three exceptions made sense. Before we can explain why, we need to get a little bit into how query engines read Parquet files.
Reading Parquets
Remember we mentioned Iceberg metadata? One kind of metadata that Iceberg saves is a list of manifest entries. Each entry contains information about a data file (in our case, Parquet files) that is part of the table.
Among other things, it contains the path of the file and the size in bytes of the file. Parquet is a very efficient format designed to allow engines to read the minimum amount of data required for their queries. At the end of the Parquet file, there's a footer containing information about the data in the file. This footer helps engines read only what they need or even skip the file completely.
Data files are usually big, they contain the actual data! Therefore, query engines want to scan as few data files as possible, and as little data from each file as they can. The Parquet footer helps them accomplish this.
To optimize performance, query engines like Spark don't immediately read the entire data file. First, they read only the footer to determine if the file is relevant to their query and which parts they should fetch. To locate this footer, the engine needs to know where in the file it begins. It finds this position using the file size information stored in the manifest entry of the Iceberg metadata. The engine uses the size as specified in the manifest entry to locate the footer starting from the end of the file.
Back to the Issue
In our case, since files were silently overwritten, some metadata entries were wrong! The old metadata entries contained incorrect information about file size. The actual file could be larger or smaller than what was recorded in the metadata. This discrepancy caused engines to look for the Parquet footer in the wrong place, explaining all the errors we encountered.
- If the size in the metadata was more than the actual size - the engine requested an invalid range outside the bounds of the file
- If the size in the metadata was less than the actual size - the engine tried to read the footer from the middle of the file instead of from its end, resulting in unparsable information
Here is a simplified visualization of the information in the metadata, versus the actual data:

Wrapping Up
The issue we encountered is not new. But despite a fix that was released 1.5 years ago, it is still widespread in production environments.
- This is a severe issue. It caused data corruption, which rendered tables practically unreadable.
- It is also a data integrity issue. When it manifested, table data was permanently lost. Without continuous monitoring of table metadata evolution, it is not even possible to know exactly which and how much data was impacted.
- An Iceberg version containing this bug is the default Iceberg version bundled with EMR 6.x. Though EMR 7 has been available for some time, upgrading Spark versions is hard. As a result, EMR 6.x remains widely used.
Are you Affected?
- If you're using Spark Streaming with Iceberg ≤1.5.0, we recommend upgrading your Iceberg version.
- If you're using EMR 6.x and cannot upgrade to EMR 7.x, you can still upgrade your Iceberg version. Rather than using the bundled Iceberg JAR, run your streaming applications with a newer version of Iceberg. You'll benefit from many other improvements too!
Takeaways
Iceberg gives you unprecedented ownership over your own data.
With this freedom, however, comes responsibility that wasn’t there before. You are free to use whatever engine you want, but you will also have to deal with its own set of intricacies. You are free to save your data in whatever way that fits your need, but when it gets corrupted - it’s on you to debug and find a fix.
Stay Tuned
Next up in this series is a dive in into another serious Iceberg table corruption issue we encountered in the wild, stay tuned for Part 2.