Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Move FsInput creation into AvroFileReader #5556

Merged
merged 1 commit into from
May 20, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package com.nvidia.spark.rapids

import java.io.{EOFException, InputStream, IOException, OutputStream}
import java.net.URI
import java.nio.charset.StandardCharsets

import scala.collection.mutable
Expand All @@ -25,7 +26,10 @@ import org.apache.avro.Schema
import org.apache.avro.file.DataFileConstants._
import org.apache.avro.file.SeekableInput
import org.apache.avro.io.{BinaryData, BinaryDecoder, DecoderFactory}
import org.apache.avro.mapred.FsInput
import org.apache.commons.io.output.{CountingOutputStream, NullOutputStream}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path

private[rapids] class AvroSeekableInputStream(in: SeekableInput) extends InputStream
with SeekableInput {
Expand Down Expand Up @@ -434,13 +438,21 @@ class AvroDataFileReader(si: SeekableInput) extends AvroFileReader(si) {

}

object AvroFileReader {
object AvroFileReader extends Arm {

def openMetaReader(si: SeekableInput): AvroMetaFileReader = {
new AvroMetaFileReader(si)
def openMetaReader(filePath: String, conf: Configuration): AvroMetaFileReader = {
closeOnExcept(openFile(filePath, conf)) { si =>
new AvroMetaFileReader(si)
}
}

def openDataReader(filePath: String, conf: Configuration): AvroDataFileReader = {
closeOnExcept(openFile(filePath, conf)) { si =>
new AvroDataFileReader(si)
}
}

def openDataReader(si: SeekableInput): AvroDataFileReader = {
new AvroDataFileReader(si)
private def openFile(filePath: String, conf: Configuration): SeekableInput = {
new FsInput(new Path(new URI(filePath)), conf)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ import com.nvidia.spark.rapids.GpuMetric.{GPU_DECODE_TIME, NUM_OUTPUT_BATCHES, P
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import org.apache.avro.Schema
import org.apache.avro.file.DataFileConstants.SYNC_SIZE
import org.apache.avro.mapred.FsInput
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FSDataInputStream, Path}

Expand Down Expand Up @@ -701,9 +700,7 @@ class GpuMultiFileCloudAvroPartitionReader(
*/
private def doRead(): HostMemoryBuffersWithMetaDataBase = {
val startingBytesRead = fileSystemBytesRead()
val in = new FsInput(new Path(new URI(partFile.filePath)), config)
val reader = closeOnExcept(in) { _ => AvroFileReader.openDataReader(in) }
withResource(reader) { _ =>
withResource(AvroFileReader.openDataReader(partFile.filePath, config)) { reader =>
// Go to the start of the first block after the start position
reader.sync(partFile.start)
if (!reader.hasNextBlock || isDone) {
Expand Down Expand Up @@ -959,9 +956,7 @@ case class AvroFileFilterHandler(

def filterBlocks(partFile: PartitionedFile): AvroBlockMeta = {
if (ignoreExtension || partFile.filePath.endsWith(".avro")) {
val in = new FsInput(new Path(new URI(partFile.filePath)), hadoopConf)
val reader = closeOnExcept(in) { _ => AvroFileReader.openMetaReader(in) }
withResource(reader) { _ =>
withResource(AvroFileReader.openMetaReader(partFile.filePath, hadoopConf)) { reader =>
// Get blocks only belong to this split
reader.sync(partFile.start)
val partBlocks = reader.getPartialBlocks(partFile.start + partFile.length)
Expand Down