This is an automated email from the ASF dual-hosted git repository.
viirya pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 6bcd09553bb6 [SPARK-53784] Additional Source APIs needed to support
RTM execution
6bcd09553bb6 is described below
commit 6bcd09553bb675d936f4652d8ad16ae9a14ff5b6
Author: Jerry Peng <[email protected]>
AuthorDate: Tue Oct 7 09:42:43 2025 -0700
[SPARK-53784] Additional Source APIs needed to support RTM execution
### What changes were proposed in this pull request?
Described in the jira ticket:
https://issues.apache.org/jira/browse/SPARK-53784
Here is an example for reference of how the API is used to craft a
in-memory source that supports RTM:
[sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/LowLatencyMemoryStream.scala
](https://github.com/apache/spark/pull/52502/files#diff-b49f6f9fba5c2683792c52015219d3f29ed683cc2e676da8781c0e8b84a5e272)
### Why are the changes needed?
Needed to support Real-time Mode time based batch execution in sources.
### Does this PR introduce _any_ user-facing change?
Yes, adds additional APIs needed for sources to be able integrate with
Real-time mode.
### How was this patch tested?
Not applicable since only interfaces are introduced
### Was this patch authored or co-authored using generative AI tooling?
no
Closes #52501 from jerrypeng/SPARK-53784.
Authored-by: Jerry Peng <[email protected]>
Signed-off-by: Liang-Chi Hsieh <[email protected]>
---
.../read/streaming/SupportsRealTimeMode.java | 47 ++++++++++++
.../read/streaming/SupportsRealTimeRead.java | 86 ++++++++++++++++++++++
2 files changed, 133 insertions(+)
diff --git
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsRealTimeMode.java
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsRealTimeMode.java
new file mode 100644
index 000000000000..da2127d44b4e
--- /dev/null
+++
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsRealTimeMode.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.read.streaming;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.read.InputPartition;
+
+/**
+ * A {@link MicroBatchStream} for streaming queries with real time mode.
+ *
+ */
+@Evolving
+public interface SupportsRealTimeMode {
+ /**
+ * Returns a list of {@link InputPartition input partitions} given the
start offset. Each
+ * {@link InputPartition} represents a data split that can be processed by
one Spark task. The
+ * number of input partitions returned here is the same as the number of
RDD partitions
+ * this scan outputs.
+ */
+ InputPartition[] planInputPartitions(Offset start);
+
+ /**
+ * Merge partitioned offsets coming from {@link SupportsRealTimeMode}
instances
+ * for each partition to a single global offset.
+ */
+ Offset mergeOffsets(PartitionOffset[] offsets);
+
+ /**
+ * Called during logical planning to inform the source if it's in real
time mode
+ */
+ default void prepareForRealTimeMode() {}
+}
diff --git
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsRealTimeRead.java
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsRealTimeRead.java
new file mode 100644
index 000000000000..5bed945432c9
--- /dev/null
+++
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsRealTimeRead.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.read.streaming;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.read.PartitionReader;
+
+/**
+ * A variation on {@link PartitionReader} for use with low latency streaming
processing.
+ *
+ */
+@Evolving
+public interface SupportsRealTimeRead<T> extends PartitionReader<T> {
+
+ /**
+ * A class to represent the status of a record to be read as the return
type of nextWithTimeout.
+ * It contains whether the next record is available and the ingestion time
of the record
+ * if the source connector provided relevant info. A list of source
connector that has ingestion
+ * time is listed below:
+ * - Kafka when the record timestamp type is LogAppendTime
+ * - Kinesis has ApproximateArrivalTimestamp
+ */
+ class RecordStatus {
+ private final boolean hasRecord;
+ private final Optional<Long> recArrivalTime;
+
+ private RecordStatus(boolean hasRecord, Optional<Long> recArrivalTime)
{
+ this.hasRecord = hasRecord;
+ this.recArrivalTime = recArrivalTime;
+ }
+
+ // Public factory methods to control instance creation
+ public static RecordStatus newStatusWithoutArrivalTime(boolean
hasRecord) {
+ return new RecordStatus(hasRecord, Optional.empty());
+ }
+
+ public static RecordStatus newStatusWithArrivalTimeMs(Long
recArrivalTime) {
+ return new RecordStatus(true, Optional.of(recArrivalTime));
+ }
+
+ public boolean hasRecord() {
+ return hasRecord;
+ }
+
+ public Optional<Long> recArrivalTime() {
+ return recArrivalTime;
+ }
+ }
+
+ /**
+ * Get the offset of the next record, or the start offset if no records
have been read.
+ * <p>
+ * The execution engine will call this method along with get() to keep
track of the current
+ * offset. When a task ends, the offset in each partition will be passed
back to the driver.
+ * They will be used as the start offsets of the next batch.
+ */
+ PartitionOffset getOffset();
+
+ /**
+ * Alternative function to be called than next(), that proceed to the next
record. The different
+ * from next() is that, if there is no more records, the call needs to
keep waiting until
+ * the timeout.
+ * @param timeout if no result is available after this timeout
(milliseconds), return
+ * @return {@link RecordStatus} describing whether a record is available
and its arrival time
+ * @throws IOException
+ */
+ RecordStatus nextWithTimeout(Long timeout) throws IOException;
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]