Hi John, The issue looks similar to https://github.com/apache/iceberg/issues/1654, a bug which we came across in avro when using s3a. The fix/workaround is mentioned in the link as well.
Best, Mouli On Tue, Nov 17, 2020 at 4:24 AM John Clara <john.anthony.cl...@gmail.com> wrote: > Hi, > > I was wondering if someone could give me some pointers on this line: > https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/avro/AvroIO.java#L53 > > Some threads keep getting stuck trying to read the manifest list on > commit. From debug logs, it looks like we're reading the avro file multiple > times? > > From a stack trace, it looks like we're using the AvroInputStreamAdapter. > Our metadata table io should be a HadoopFileIO (with s3a). Our classpath > has all the specified files but org.apache.avro.file.SeekableInput isn't > changed. > > Should it be relocated? Should we be trying to use the AvroFSInput? Is > the boolean just named wrong? > > (stack trace): > "dse-ingester-system-akka.actor.default-dispatcher-5" #39 prio=5 os_prio=0 > tid=0x00007fbbec00b000 nid=0x40 runnable [0x00007fbc4c52b000] > java.lang.Thread.State: RUNNABLE > at > org.apache.iceberg.avro.AvroIO$AvroInputStreamAdapter.read(AvroIO.java:120) > at > org.apache.avro.file.DataFileReader.openReader(DataFileReader.java:61) > at > org.apache.iceberg.avro.AvroIterable.newFileReader(AvroIterable.java:100) > at org.apache.iceberg.avro.AvroIterable.iterator(AvroIterable.java:77) > at org.apache.iceberg.avro.AvroIterable.iterator(AvroIterable.java:37) > at > org.apache.iceberg.relocated.com.google.common.collect.Iterables.addAll(Iterables.java:320) > at > org.apache.iceberg.relocated.com.google.common.collect.Lists.newLinkedList(Lists.java:237) > at org.apache.iceberg.ManifestLists.read(ManifestLists.java:46) > at > org.apache.iceberg.BaseSnapshot.cacheManifests(BaseSnapshot.java:127) > at org.apache.iceberg.BaseSnapshot.allManifests(BaseSnapshot.java:141) > at org.apache.iceberg.FastAppend.apply(FastAppend.java:142) > at org.apache.iceberg.SnapshotProducer.apply(SnapshotProducer.java:149) > at > org.apache.iceberg.SnapshotProducer.lambda$commit$2(SnapshotProducer.java:262) > at > org.apache.iceberg.SnapshotProducer$$Lambda$1641/2145060236.run(Unknown > Source) > at > org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:404) > at > org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:213) > at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:197) > at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:189) > at > org.apache.iceberg.SnapshotProducer.commit(SnapshotProducer.java:261) > at > com.box.dataplatform.iceberg.client.DPIcebergClientImpl.commitBatch(DPIcebergClientImpl.java:85) > ... > Locked ownable synchronizers: > - None > > > Thanks, > > John >