Skip to content
Closed
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 @@ -174,23 +174,29 @@ private[sql] object AvroUtils extends Logging {
private[this] var currentRow: Option[InternalRow] = None

def hasNextRow: Boolean = {
do {
while (!completed && currentRow.isEmpty) {
val r = fileReader.hasNext && !fileReader.pastSync(stopPosition)
if (!r) {
fileReader.close()
completed = true
currentRow = None
} else {
val record = fileReader.next()
// the row must be deserialized in hasNextRow, because AvroDeserializer#deserialize
// potentially filters rows
currentRow = deserializer.deserialize(record).asInstanceOf[Option[InternalRow]]
}
} while (!completed && currentRow.isEmpty)

}
currentRow.isDefined
}

def nextRow: InternalRow = {
currentRow.getOrElse {
if (currentRow.isEmpty) {
hasNextRow
}
val returnRow = currentRow
currentRow = None // free up hasNextRow to consume more Avro records, if not exhausted
returnRow.getOrElse {
throw new NoSuchElementException("next on empty iterator")
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.avro

import java.io._
import java.net.URL
import java.net.{URI, URL}
import java.nio.file.{Files, Paths, StandardCopyOption}
import java.sql.{Date, Timestamp}
import java.util.{Locale, UUID}
Expand All @@ -31,16 +31,20 @@ import org.apache.avro.Schema.Type._
import org.apache.avro.file.{DataFileReader, DataFileWriter}
import org.apache.avro.generic.{GenericData, GenericDatumReader, GenericDatumWriter, GenericRecord}
import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed}
import org.apache.avro.mapred.FsInput
import org.apache.commons.io.FileUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path

import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf, SparkException, SparkUpgradeException}
import org.apache.spark.sql._
import org.apache.spark.sql.TestingUDT.IntervalData
import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters}
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.plans.logical.Filter
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, LA, UTC}
import org.apache.spark.sql.execution.{FormattedMode, SparkPlan}
import org.apache.spark.sql.execution.datasources.{CommonFileDataSourceSuite, DataSource, FilePartition}
import org.apache.spark.sql.execution.datasources.{CommonFileDataSourceSuite, DataSource, FilePartition, PartitionedFile}
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -1836,6 +1840,24 @@ abstract class AvroSuite
}
}
}

test("SPARK-33314: RowReader doesn't over-consume when hasNextRow called twice") {
withTempPath { dir =>
Seq((1), (2), (3))
.toDF("index")
.write
.format("avro")
.save(dir.getCanonicalPath)
val df = spark
.read
.format("avro")
.load(dir.getCanonicalPath)
.orderBy("index")

checkAnswer(df,
Seq(Row(1), Row(2), Row(3)))
}
}
}

class AvroV1Suite extends AvroSuite {
Expand Down Expand Up @@ -2005,3 +2027,61 @@ class AvroV2Suite extends AvroSuite with ExplainSuiteHelper {
}
}
}

class AvroRowReaderSuite
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@bersprockets @HyukjinKwon I just noticed recently that this suite is in AvroSuite.scala. Are there any specific reasons to not place it to AvroRowReaderSuite.scala?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think so. Feel free to move it there.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please, review the PR #32607

extends QueryTest
with SharedSparkSession {

import testImplicits._

override protected def sparkConf: SparkConf =
super
.sparkConf
.set(SQLConf.USE_V1_SOURCE_LIST, "") // need this for BatchScanExec

test("SPARK-33314: hasNextRow and nextRow properly handle consecutive calls") {
withTempPath { dir =>
Seq((1), (2), (3))
.toDF("value")
.coalesce(1)
.write
.format("avro")
.save(dir.getCanonicalPath)

val df = spark.read.format("avro").load(dir.getCanonicalPath)
val fileScan = df.queryExecution.executedPlan collectFirst {
case BatchScanExec(_, f: AvroScan) => f
}
val filePath = fileScan.get.fileIndex.inputFiles(0)
val fileSize = new File(new URI(filePath)).length
val in = new FsInput(new Path(new URI(filePath)), new Configuration())
val reader = DataFileReader.openReader(in, new GenericDatumReader[GenericRecord]())

val it = new Iterator[InternalRow] with AvroUtils.RowReader {
override val fileReader = reader
override val deserializer = new AvroDeserializer(
reader.getSchema,
StructType(new StructField("value", IntegerType, true) :: Nil),
CORRECTED,
new NoopFilters)
override val stopPosition = fileSize

override def hasNext: Boolean = hasNextRow

override def next: InternalRow = nextRow
}
assert(it.hasNext == true)
assert(it.next.getInt(0) == 1)
// test no intervening next
assert(it.hasNext == true)
assert(it.hasNext == true)
// test no intervening hasNext
assert(it.next.getInt(0) == 2)
assert(it.next.getInt(0) == 3)
assert(it.hasNext == false)
assertThrows[NoSuchElementException] {
it.next
}
}
}
}