apache / iceberg

Apache Iceberg
https://iceberg.apache.org/
Apache License 2.0
6.34k stars 2.19k forks source link

Reading manifest Avro hangs when S3A inputstream returns less bytes than requested #1654

Closed mickjermsurawong-stripe closed 3 years ago

mickjermsurawong-stripe commented 3 years ago

Observation: Spark job intermittently hangs with driver thread stuck in reading avro file with S3A inputstream, according to Thread dump here.

java.io.DataInputStream.read(DataInputStream.java:149)
org.apache.iceberg.hadoop.HadoopStreams$HadoopSeekableInputStream.read(HadoopStreams.java:112)
org.apache.iceberg.avro.AvroIO$AvroInputStreamAdapter.read(AvroIO.java:106)
org.apache.iceberg.shaded.org.apache.avro.file.DataFileReader.openReader(DataFileReader.java:61)
org.apache.iceberg.avro.AvroIterable.newFileReader(AvroIterable.java:94)
org.apache.iceberg.avro.AvroIterable.iterator(AvroIterable.java:77)
org.apache.iceberg.avro.AvroIterable.iterator(AvroIterable.java:37)
org.apache.iceberg.relocated.com.google.common.collect.Iterables.addAll(Iterables.java:320)
org.apache.iceberg.relocated.com.google.common.collect.Lists.newLinkedList(Lists.java:237)
org.apache.iceberg.ManifestLists.read(ManifestLists.java:46)
org.apache.iceberg.BaseSnapshot.cacheManifests(BaseSnapshot.java:127)
org.apache.iceberg.BaseSnapshot.dataManifests(BaseSnapshot.java:149)
org.apache.iceberg.DataTableScan.planFiles(DataTableScan.java:74)
org.apache.iceberg.BaseTableScan.planFiles(BaseTableScan.java:212)
org.apache.iceberg.DataTableScan.planFiles(DataTableScan.java:28)
org.apache.iceberg.BaseTableScan.planTasks(BaseTableScan.java:245)
org.apache.iceberg.DataTableScan.planTasks(DataTableScan.java:28)
org.apache.iceberg.spark.source.Reader.tasks(Reader.java:365)

Affected code: DataFileReader.openReader https://github.com/apache/iceberg/blob/f17879c426c2e3c5fa40f17dfe58633bc866f1c9/core/src/main/java/org/apache/iceberg/avro/AvroIterable.java#L98-L105 Root cause: Avro reader checking header has incorrect logic and will get stuck in infinite loop when S3A input stream reads fewer bytes than requested. This has been raised with Avro https://issues.apache.org/jira/browse/AVRO-2944

Current: https://github.com/apache/avro/blob/328c539afc77da347ec52be1e112a6a7c371143b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader.java#L61-L62

for (int c = 0; c < magic.length; c = in.read(magic, c, magic.length - c)) {}

Fix in Avro:

for (int c = 0; c < magic.length; c += in.read(magic, c, magic.length - c)) {}

Iceberg change needed: There is PR for Avro https://github.com/apache/avro/pull/969 and I'm wondering if Iceberg can take this Avro patch in, once merged.

mickjermsurawong-stripe commented 3 years ago

The Avro fix https://github.com/apache/avro/pull/969 has been merged and it's in version 1.10. Ideally the fix is back-ported to older versions too. I've sent a note to Avro dev to get a sense of how pressing is this problem to others.

If this would help others, we made a patch to our internal Iceberg to avoid the hang instead. Specifically, we re-implemented the thin logic of checking the Avro header in Iceberg, and do nothing to Avro.

rdblue commented 3 years ago

@mickjermsurawong-stripe, what does the Iceberg patch look like?

mickjermsurawong-stripe commented 3 years ago

Hi ryan, it's really not a great patch we made. From the original static method openReader which originally does two things:

We simply change any invocation in Iceberg for DataFileReader.openReader to our new class AvroPatch2944DataFileReader.openReader

public class AvroPatch2944DataFileReader {
  public static <D> FileReader<D> openReader(SeekableInput in, DatumReader<D> reader) throws IOException {
      // read header with corrected pointer
      // instantiate DataFileReader
  }
}

The alternative is to simply skip checking header. Since the avro files read here is always written by the Iceberg framework, i think it is safe to simply call constructor of the file reader for new version (instead of 1.2). I'd prefer this for upstream Iceberg.

For our internal fork, we don't want to make the same assumption, so we took a more conservative approach, maintaining the same logic with re-implementation

johnclara commented 3 years ago

@mickjermsurawong-stripe we just added a less clean version of the patch and it immediately fixed our dev ingestion service.

We're able to reproduce this very reliably (but not in a clean environment) if someone has any other suggestions for fixes

mickjermsurawong-stripe commented 3 years ago

Thanks @johnclara for confirming the problem here. We have also been running more smoothly after our internal patch.

Just would like to bump this if upgrading avro to version 1.10.1 (to include this patch) should be cosidered for next iceberg release. One complication could be that hadoop still uses avro 1.7 (iceberg currently uses avro 1.9)

cc @rdblue

grantatspothero commented 3 years ago

Confirmed this issue is still a problem and causes iceberg task planning to hang forever.

Upgrading to avro 1.10.1 resolves the hanging problem, but as @mickjermsurawong-stripe mentioned there might be other side effects of upgrading that require testing.

This bug also affects trino's iceberg plugin: https://github.com/trinodb/trino/blame/master/plugin/trino-iceberg/pom.xml#L135-L145

They exclude the avro dep to pin to a specific version, but the version they pin to is 1.9.2: https://github.com/trinodb/trino/blob/master/pom.xml#L1305-L1309

grantatspothero commented 3 years ago

Ended up doing the same thing @mickjermsurawong-stripe described above, forking iceberg and copying the static method from avro into iceberg: https://github.com/apache/iceberg/compare/apache-iceberg-0.11.0...grantatspothero:0.11.0-avro-patch?expand=1

This resolves the issue without having to upgrade avro and hope that interops well with trino/spark.

rdblue commented 3 years ago

I think we should plan on upgrading to Avro in #1648. That should fix this in the next release, 0.12.0.

rdblue commented 3 years ago

This is in the 0.12.0 release so I'll close this.