Skip to content

Commit fe73cb4

Browse files
cloud-fangatorsmile
authored andcommitted
[SPARK-23317][SQL] rename ContinuousReader.setOffset to setStartOffset
## What changes were proposed in this pull request? In the document of `ContinuousReader.setOffset`, we say this method is used to specify the start offset. We also have a `ContinuousReader.getStartOffset` to get the value back. I think it makes more sense to rename `ContinuousReader.setOffset` to `setStartOffset`. ## How was this patch tested? N/A Author: Wenchen Fan <[email protected]> Closes #20486 from cloud-fan/rename.
1 parent 3ff83ad commit fe73cb4

File tree

6 files changed

+7
-7
lines changed

6 files changed

+7
-7
lines changed

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ class KafkaContinuousReader(
7171
override def readSchema: StructType = KafkaOffsetReader.kafkaSchema
7272

7373
private var offset: Offset = _
74-
override def setOffset(start: ju.Optional[Offset]): Unit = {
74+
override def setStartOffset(start: ju.Optional[Offset]): Unit = {
7575
offset = start.orElse {
7676
val offsets = initialOffsets match {
7777
case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets())

sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -51,12 +51,12 @@ public interface ContinuousReader extends BaseStreamingSource, DataSourceReader
5151
* start from the first record after the provided offset, or from an implementation-defined
5252
* inferred starting point if no offset is provided.
5353
*/
54-
void setOffset(Optional<Offset> start);
54+
void setStartOffset(Optional<Offset> start);
5555

5656
/**
5757
* Return the specified or inferred start offset for this reader.
5858
*
59-
* @throws IllegalStateException if setOffset has not been called
59+
* @throws IllegalStateException if setStartOffset has not been called
6060
*/
6161
Offset getStartOffset();
6262

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -181,7 +181,7 @@ class ContinuousExecution(
181181

182182
val loggedOffset = offsets.offsets(0)
183183
val realOffset = loggedOffset.map(off => reader.deserializeOffset(off.json))
184-
reader.setOffset(java.util.Optional.ofNullable(realOffset.orNull))
184+
reader.setStartOffset(java.util.Optional.ofNullable(realOffset.orNull))
185185
new StreamingDataSourceV2Relation(newOutput, reader)
186186
}
187187

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,7 @@ class RateStreamContinuousReader(options: DataSourceOptions)
6161

6262
private var offset: Offset = _
6363

64-
override def setOffset(offset: java.util.Optional[Offset]): Unit = {
64+
override def setStartOffset(offset: java.util.Optional[Offset]): Unit = {
6565
this.offset = offset.orElse(RateStreamSourceV2.createInitialOffset(numPartitions, creationTime))
6666
}
6767

sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -160,7 +160,7 @@ class RateSourceV2Suite extends StreamTest {
160160
test("continuous data") {
161161
val reader = new RateStreamContinuousReader(
162162
new DataSourceOptions(Map("numPartitions" -> "2", "rowsPerSecond" -> "20").asJava))
163-
reader.setOffset(Optional.empty())
163+
reader.setStartOffset(Optional.empty())
164164
val tasks = reader.createDataReaderFactories()
165165
assert(tasks.size == 2)
166166

sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ case class FakeReader() extends MicroBatchReader with ContinuousReader {
4343
def readSchema(): StructType = StructType(Seq())
4444
def stop(): Unit = {}
4545
def mergeOffsets(offsets: Array[PartitionOffset]): Offset = RateStreamOffset(Map())
46-
def setOffset(start: Optional[Offset]): Unit = {}
46+
def setStartOffset(start: Optional[Offset]): Unit = {}
4747

4848
def createDataReaderFactories(): java.util.ArrayList[DataReaderFactory[Row]] = {
4949
throw new IllegalStateException("fake source - cannot actually read")

0 commit comments

Comments
 (0)