Details

    Description

      When using S3AFileSystem to read Parquet files a specific set of circumstances causes an  EOFException that is not thrown when reading the same file from local disk

      Note this has only been observed under specific circumstances:
        - when the reader is doing a projection (will cause it to do a seek backwards and put the filesystem into random mode)
       - when the file is larger than the readahead buffer size
       - when the seek behavior of the Parquet reader causes the reader to seek towards the end of the current input stream without reopening, such that the next read on the currently open stream will read past the end of the currently open stream.

      Exception from Parquet reader is as follows:

      Caused by: java.io.EOFException: Reached the end of stream with 51 bytes left to read
       at org.apache.parquet.io.DelegatingSeekableInputStream.readFully(DelegatingSeekableInputStream.java:104)
       at org.apache.parquet.io.DelegatingSeekableInputStream.readFullyHeapBuffer(DelegatingSeekableInputStream.java:127)
       at org.apache.parquet.io.DelegatingSeekableInputStream.readFully(DelegatingSeekableInputStream.java:91)
       at org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:1174)
       at org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:805)
       at org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:127)
       at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:222)
       at org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
       at org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormatBase.fetchNext(HadoopInputFormatBase.java:206)
       at org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormatBase.reachedEnd(HadoopInputFormatBase.java:199)
       at org.apache.flink.runtime.operators.DataSourceTask.invoke(DataSourceTask.java:190)
       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
       at java.lang.Thread.run(Thread.java:748)
      

      The following example program generate the same root behavior (sans finding a Parquet file that happens to trigger this condition) by purposely reading past the already active readahead range on any file >= 1029 bytes in size.. 

      final Configuration conf = new Configuration();
      conf.set("fs.s3a.readahead.range", "1K");
      conf.set("fs.s3a.experimental.input.fadvise", "random");
      
      final FileSystem fs = FileSystem.get(path.toUri(), conf);
      // forward seek reading across readahead boundary
      try (FSDataInputStream in = fs.open(path)) {
          final byte[] temp = new byte[5];
          in.readByte();
          in.readFully(1023, temp); // <-- works
      }
      // forward seek reading from end of readahead boundary
      try (FSDataInputStream in = fs.open(path)) {
       final byte[] temp = new byte[5];
       in.readByte();
       in.readFully(1024, temp); // <-- throws EOFException
      }
      

       

      Attachments

        1. HADOOP-16109-branch-3.1-003.patch
          15 kB
          Steve Loughran

        Issue Links

          Activity

            People

              stevel@apache.org Steve Loughran
              Christianson Dave Christianson
              Votes:
              0 Vote for this issue
              Watchers:
              9 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: