HeartSaVioR commented on a change in pull request #30221: URL: https://github.com/apache/spark/pull/30221#discussion_r515822838
########## File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala ########## @@ -171,9 +171,15 @@ private[sql] object AvroUtils extends Logging { protected val stopPosition: Long private[this] var completed = false + private[this] var interveningNext = true + private[this] var prevHasNextRow = false private[this] var currentRow: Option[InternalRow] = None def hasNextRow: Boolean = { + if (!interveningNext) { + // until a row is consumed, return previous result of hasNextRow + return prevHasNextRow + } Review comment: I also feel @viirya 's suggestion would be simpler. In addition, looks like the implementation didn't respect the Iterator's contracts - calling `hasNextRow` shouldn't be prerequisite to call `nextRow`. Below code would fix the original issue (as this code passes the new test), as well as it would also work for the case only `nextRow` is called with handling NoSuchElementException. ``` def hasNextRow: Boolean = { if (currentRow.isDefined) { return true } while (!completed && currentRow.isEmpty) { val r = fileReader.hasNext && !fileReader.pastSync(stopPosition) if (!r) { fileReader.close() completed = true currentRow = None } else { val record = fileReader.next() currentRow = deserializer.deserialize(record).asInstanceOf[Option[InternalRow]] } } currentRow.isDefined } def nextRow: InternalRow = { if (currentRow.isEmpty) { if (!hasNextRow) { throw new NoSuchElementException("next on empty iterator") } } val row = currentRow.get currentRow = None row } ``` ########## File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala ########## @@ -171,9 +171,15 @@ private[sql] object AvroUtils extends Logging { protected val stopPosition: Long private[this] var completed = false + private[this] var interveningNext = true + private[this] var prevHasNextRow = false private[this] var currentRow: Option[InternalRow] = None def hasNextRow: Boolean = { + if (!interveningNext) { + // until a row is consumed, return previous result of hasNextRow + return prevHasNextRow + } Review comment: I also feel @viirya 's suggestion would be simpler. In addition, looks like the implementation didn't respect the Iterator's contracts - calling `hasNextRow` shouldn't be prerequisite to call `nextRow`. Below code would fix the original issue (as this code passes the new test), as well as it would also work for the case only `nextRow` is called with handling NoSuchElementException. ``` def hasNextRow: Boolean = { if (currentRow.isDefined) { return true } while (!completed && currentRow.isEmpty) { val r = fileReader.hasNext && !fileReader.pastSync(stopPosition) if (!r) { fileReader.close() completed = true currentRow = None } else { val record = fileReader.next() currentRow = deserializer.deserialize(record).asInstanceOf[Option[InternalRow]] } } currentRow.isDefined } def nextRow: InternalRow = { if (currentRow.isEmpty) { if (!hasNextRow) { throw new NoSuchElementException("next on empty iterator") } } val row = currentRow.get currentRow = None row } ``` ########## File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala ########## @@ -171,9 +171,15 @@ private[sql] object AvroUtils extends Logging { protected val stopPosition: Long private[this] var completed = false + private[this] var interveningNext = true + private[this] var prevHasNextRow = false private[this] var currentRow: Option[InternalRow] = None def hasNextRow: Boolean = { + if (!interveningNext) { + // until a row is consumed, return previous result of hasNextRow + return prevHasNextRow + } Review comment: I also feel @viirya 's suggestion would be simpler. In addition, looks like the implementation didn't respect the Iterator's contracts - calling `hasNextRow` shouldn't be prerequisite to call `nextRow`. Below code would fix the original issue (as this code passes the new test), as well as it would also work for the case which only calls `nextRow` with handling `NoSuchElementException`. ``` def hasNextRow: Boolean = { if (currentRow.isDefined) { return true } while (!completed && currentRow.isEmpty) { val r = fileReader.hasNext && !fileReader.pastSync(stopPosition) if (!r) { fileReader.close() completed = true currentRow = None } else { val record = fileReader.next() currentRow = deserializer.deserialize(record).asInstanceOf[Option[InternalRow]] } } currentRow.isDefined } def nextRow: InternalRow = { if (currentRow.isEmpty) { if (!hasNextRow) { throw new NoSuchElementException("next on empty iterator") } } val row = currentRow.get currentRow = None row } ``` ########## File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala ########## @@ -171,9 +171,15 @@ private[sql] object AvroUtils extends Logging { protected val stopPosition: Long private[this] var completed = false + private[this] var interveningNext = true + private[this] var prevHasNextRow = false private[this] var currentRow: Option[InternalRow] = None def hasNextRow: Boolean = { + if (!interveningNext) { + // until a row is consumed, return previous result of hasNextRow + return prevHasNextRow + } Review comment: I also feel @viirya 's suggestion would be simpler. In addition, looks like the implementation didn't respect the Iterator's contracts - calling `hasNextRow` explicitly shouldn't be prerequisite to call `nextRow`. Below code would fix the original issue (as this code passes the new test), as well as it would also work for the case which only calls `nextRow` with handling `NoSuchElementException`. ``` def hasNextRow: Boolean = { if (currentRow.isDefined) { return true } while (!completed && currentRow.isEmpty) { val r = fileReader.hasNext && !fileReader.pastSync(stopPosition) if (!r) { fileReader.close() completed = true currentRow = None } else { val record = fileReader.next() currentRow = deserializer.deserialize(record).asInstanceOf[Option[InternalRow]] } } currentRow.isDefined } def nextRow: InternalRow = { if (currentRow.isEmpty) { if (!hasNextRow) { throw new NoSuchElementException("next on empty iterator") } } val row = currentRow.get currentRow = None row } ``` ########## File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala ########## @@ -171,9 +171,15 @@ private[sql] object AvroUtils extends Logging { protected val stopPosition: Long private[this] var completed = false + private[this] var interveningNext = true + private[this] var prevHasNextRow = false private[this] var currentRow: Option[InternalRow] = None def hasNextRow: Boolean = { + if (!interveningNext) { + // until a row is consumed, return previous result of hasNextRow + return prevHasNextRow + } Review comment: I also feel @viirya 's suggestion would be simpler. In addition, looks like the implementation didn't respect the Iterator's contracts - calling `hasNextRow` explicitly shouldn't be prerequisite to call `nextRow`. Below code would fix the original issue (as this code passes the new test), as well as it would also work for the case which only calls `nextRow` with handling `NoSuchElementException`. ``` def hasNextRow: Boolean = { while (!completed && currentRow.isEmpty) { val r = fileReader.hasNext && !fileReader.pastSync(stopPosition) if (!r) { fileReader.close() completed = true currentRow = None } else { val record = fileReader.next() currentRow = deserializer.deserialize(record).asInstanceOf[Option[InternalRow]] } } currentRow.isDefined } def nextRow: InternalRow = { if (currentRow.isEmpty) { if (!hasNextRow) { throw new NoSuchElementException("next on empty iterator") } } val row = currentRow.get currentRow = None row } ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org