Skip to content
Open
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
29 changes: 29 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -3832,6 +3832,12 @@
],
"sqlState" : "42601"
},
"INVALID_STREAMING_REAL_TIME_MODE_TRIGGER_INTERVAL" : {
"message" : [
"The real-time trigger interval is set to <interval> ms. This is less than the <minBatchDuration> ms minimum specified by spark.sql.streaming.realTimeMode.minBatchDuration."
],
"sqlState" : "22023"
},
"INVALID_SUBQUERY_EXPRESSION" : {
"message" : [
"Invalid subquery:"
Expand Down Expand Up @@ -5528,6 +5534,29 @@
],
"sqlState" : "XXKST"
},
"STREAMING_REAL_TIME_MODE" : {
"message" : [
"Streaming real-time mode has the following limitation:"
],
"subClass" : {
"ASYNC_PROGRESS_TRACKING_NOT_SUPPORTED" : {
"message" : [
"Async progress tracking is not supported in real-time mode. Set option asyncProgressTrackingEnabled to false and retry your query."
]
},
"IDENTICAL_SOURCES_IN_UNION_NOT_SUPPORTED" : {
"message" : [
"Real-time mode does not support union on two or more identical streaming data sources in a single query. This includes scenarios such as referencing the same source DataFrame more than once, or using two data sources with identical configurations for some sources. For Kafka, avoid reusing the same DataFrame and create different ones. Sources provided in the query: <sources>"
]
},
"INPUT_STREAM_NOT_SUPPORTED" : {
"message" : [
"The input stream <className> is not supported in Real-time Mode."
]
}
},
"sqlState" : "0A000"
},
"STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA" : {
"message" : [
"Streaming stateful operator name does not match with the operator in state metadata. This likely to happen when user adds/removes/changes stateful operator of existing streaming query.",
Expand Down
5 changes: 5 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3155,6 +3155,11 @@ object SparkContext extends Logging {
/** Separator of tags in SPARK_JOB_TAGS property */
private[spark] val SPARK_JOB_TAGS_SEP = ","

def isDriver(executorId: String): Boolean = {
// Check for null to match the behavior of executorId == DRIVER_IDENTIFIER
executorId != null && executorId.startsWith(DRIVER_IDENTIFIER)
}

// Same rules apply to Spark Connect execution tags, see ExecuteHolder.throwIfInvalidTag
private[spark] def throwIfInvalidTag(tag: String) = {
if (tag == null) {
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/util/Clock.scala
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ private[spark] trait Clock {
/**
* A clock backed by the actual time from the OS as reported by the `System` API.
*/
private[spark] class SystemClock extends Clock {
private[spark] class SystemClock extends Clock with Serializable {

val minPollTime = 25L

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,8 @@ case class StreamingDataSourceV2Relation(
catalog: Option[CatalogPlugin],
identifier: Option[Identifier],
options: CaseInsensitiveStringMap,
metadataPath: String)
metadataPath: String,
realTimeModeDuration: Option[Long] = None)
extends DataSourceV2RelationBase(table, output, catalog, identifier, options) {

override def isStreaming: Boolean = true
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2991,6 +2991,13 @@ object SQLConf {
.booleanConf
.createWithDefault(false)

val STREAMING_REAL_TIME_MODE_MIN_BATCH_DURATION = buildConf(
"spark.sql.streaming.realTimeMode.minBatchDuration")
.doc("The minimum long-running batch duration in milliseconds for real-time mode.")
.version("4.1.0")
.timeConf(TimeUnit.MILLISECONDS)
.createWithDefault(5000)

val VARIABLE_SUBSTITUTE_ENABLED =
buildConf("spark.sql.variable.substitute")
.doc("This enables substitution using syntax like `${var}`, `${system:var}`, " +
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.internal.connector;

import java.io.Serializable;

import org.apache.spark.sql.connector.read.streaming.PartitionOffset;

/**
* Internal class for real time mode to pass partition offset from executors to the driver.
*/
private[sql] case class PartitionOffsetWithIndex(index: Long, partitionOffset: PartitionOffset)
extends Serializable;
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import javax.annotation.concurrent.GuardedBy
import scala.collection.mutable
import scala.jdk.CollectionConverters._

import org.apache.spark.SparkIllegalArgumentException
import org.apache.spark.annotation.Evolving
import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.{CLASS_NAME, QUERY_ID, RUN_ID}
Expand Down Expand Up @@ -189,6 +190,21 @@ class StreamingQueryManager private[sql] (
val analyzedPlan = df.queryExecution.analyzed
df.queryExecution.assertAnalyzed()

if (trigger.isInstanceOf[RealTimeTrigger]) {
val minBatchDuration =
sparkSession.conf.get(SQLConf.STREAMING_REAL_TIME_MODE_MIN_BATCH_DURATION)
val realTimeTrigger = trigger.asInstanceOf[RealTimeTrigger]
if (realTimeTrigger.batchDurationMs < minBatchDuration) {
throw new SparkIllegalArgumentException(
errorClass = "INVALID_STREAMING_REAL_TIME_MODE_TRIGGER_INTERVAL",
messageParameters = Map(
"interval" -> realTimeTrigger.batchDurationMs.toString,
"minBatchDuration" -> minBatchDuration.toString
)
)
}
}

val dataStreamWritePlan = WriteToStreamStatement(
userSpecifiedName,
userSpecifiedCheckpointLocation,
Expand Down Expand Up @@ -216,6 +232,11 @@ class StreamingQueryManager private[sql] (
analyzedStreamWritePlan))
case _ =>
val microBatchExecution = if (useAsyncProgressTracking(extraOptions)) {
if (trigger.isInstanceOf[RealTimeTrigger]) {
throw new SparkIllegalArgumentException(
errorClass = "STREAMING_REAL_TIME_MODE.ASYNC_PROGRESS_TRACKING_NOT_SUPPORTED"
)
}
new AsyncProgressTrackingMicroBatchExecution(
sparkSession,
trigger,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import scala.collection.mutable

import org.apache.hadoop.fs.Path

import org.apache.spark.SparkException
import org.apache.spark.{SparkException, SparkIllegalArgumentException}
import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKeys.EXPR
import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable}
Expand All @@ -39,7 +39,7 @@ import org.apache.spark.sql.connector.catalog.index.SupportsIndex
import org.apache.spark.sql.connector.expressions.{FieldReference, LiteralValue}
import org.apache.spark.sql.connector.expressions.filter.{And => V2And, Not => V2Not, Or => V2Or, Predicate}
import org.apache.spark.sql.connector.read.LocalScan
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream}
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream, SupportsRealTimeMode}
import org.apache.spark.sql.connector.write.V1Write
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
import org.apache.spark.sql.execution.{FilterExec, InSubqueryExec, LeafExecNode, LocalTableScanExec, ProjectExec, RowDataSourceScanExec, SparkPlan, SparkStrategy => Strategy}
Expand Down Expand Up @@ -170,10 +170,28 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
case PhysicalOperation(p, f, r: StreamingDataSourceV2ScanRelation)
if r.startOffset.isDefined && r.endOffset.isEmpty =>

val continuousStream = r.stream.asInstanceOf[ContinuousStream]
val scanExec = ContinuousScanExec(r.output, r.scan, continuousStream, r.startOffset.get)
// initialize partitions
scanExec.inputPartitions
val scanExec = if (r.relation.realTimeModeDuration.isDefined) {
if (!r.stream.isInstanceOf[SupportsRealTimeMode]) {
throw new SparkIllegalArgumentException(
errorClass = "STREAMING_REAL_TIME_MODE.INPUT_STREAM_NOT_SUPPORTED",
messageParameters = Map("className" -> r.stream.getClass.getName)
)
}
val microBatchStream = r.stream.asInstanceOf[MicroBatchStream]
new RealTimeStreamScanExec(
r.output,
r.scan,
microBatchStream,
r.startOffset.get,
r.relation.realTimeModeDuration.get
)
} else {
val continuousStream = r.stream.asInstanceOf[ContinuousStream]
val s = ContinuousScanExec(r.output, r.scan, continuousStream, r.startOffset.get)
// initialize partitions
s.inputPartitions
s
}

// Add a Project here to make sure we produce unsafe rows.
DataSourceV2Strategy.withProjectAndFilter(p, f, scanExec, !scanExec.supportsColumnar) :: Nil
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,19 @@

package org.apache.spark.sql.execution.datasources.v2

import org.apache.spark.util.{Clock, SystemClock}
import java.util.Objects

import org.apache.spark.{SparkContext, TaskContext}
import org.apache.spark.internal.{Logging, LogKeys}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder}
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, Scan}
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset, SupportsRealTimeMode, SupportsRealTimeRead}
import org.apache.spark.sql.connector.read.streaming.SupportsRealTimeRead.RecordStatus
import org.apache.spark.sql.internal.connector.PartitionOffsetWithIndex
import org.apache.spark.util.{Clock, CollectionAccumulator, SystemClock}
import org.apache.spark.util.ArrayImplicits._

/* The singleton object to control the time in testing */
object LowLatencyClock {
Expand All @@ -37,3 +49,121 @@ object LowLatencyClock {
clock = inputClock
}
}

/**
* A wrap reader that turns a Partition Reader extending SupportsRealTimeRead to a
* normal PartitionReader and follow the task termination time `lowLatencyEndTime`, and
* report end offsets in the end to `endOffsets`.
*/
case class LowLatencyReaderWrap(
reader: SupportsRealTimeRead[InternalRow],
lowLatencyEndTime: Long,
endOffsets: CollectionAccumulator[PartitionOffsetWithIndex])
extends PartitionReader[InternalRow] {

override def next(): Boolean = {
val curTime = LowLatencyClock.getTimeMillis()
val ret = if (curTime >= lowLatencyEndTime) {
RecordStatus.newStatusWithoutArrivalTime(false)
} else {
reader.nextWithTimeout(lowLatencyEndTime - curTime)
}

if (!ret.hasRecord) {
// The way of using TaskContext.get().partitionId() to map to a partition
// may be fragile as it relies on thread locals.
endOffsets.add(
new PartitionOffsetWithIndex(TaskContext.get().partitionId(), reader.getOffset)
)
}
ret.hasRecord
}

override def get(): InternalRow = {
reader.get()
}

override def close(): Unit = {}
}

/**
* Wrapper factory that creates LowLatencyReaderWrap from reader as SupportsRealTimeRead
*/
case class LowLatencyReaderFactoryWrap(
partitionReaderFactory: PartitionReaderFactory,
lowLatencyEndTime: Long,
endOffsets: CollectionAccumulator[PartitionOffsetWithIndex])
extends PartitionReaderFactory
with Logging {
override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
val rowReader = partitionReaderFactory.createReader(partition)
assert(rowReader.isInstanceOf[SupportsRealTimeRead[InternalRow]])
logInfo(
log"Creating low latency PartitionReader, stopping at " +
log"${MDC(LogKeys.TO_TIME, lowLatencyEndTime)}"
)
LowLatencyReaderWrap(
rowReader.asInstanceOf[SupportsRealTimeRead[InternalRow]],
lowLatencyEndTime,
endOffsets
)
}
}

/**
* Physical plan node for scanning a micro-batch of data from a data source.
*/
case class RealTimeStreamScanExec(
output: Seq[Attribute],
@transient scan: Scan,
@transient stream: MicroBatchStream,
@transient start: Offset,
batchDurationMs: Long)
extends DataSourceV2ScanExecBase {

override def keyGroupedPartitioning: Option[Seq[Expression]] = None

override def ordering: Option[Seq[SortOrder]] = None

val endOffsetsAccumulator: CollectionAccumulator[PartitionOffsetWithIndex] = {
assert(stream.isInstanceOf[SupportsRealTimeMode])
SparkContext.getActive.map(_.collectionAccumulator[PartitionOffsetWithIndex]).get
}

override def equals(other: Any): Boolean = other match {
case other: RealTimeStreamScanExec =>
this.stream == other.stream &&
this.batchDurationMs == other.batchDurationMs
case _ => false
}

override def hashCode(): Int = Objects.hashCode(stream, batchDurationMs)

override lazy val readerFactory: PartitionReaderFactory = stream.createReaderFactory()

override lazy val inputPartitions: Seq[InputPartition] = {
val lls = stream.asInstanceOf[SupportsRealTimeMode]
assert(lls != null)
lls.planInputPartitions(start).toImmutableArraySeq
}

override def simpleString(maxFields: Int): String =
s"${super.simpleString(maxFields)} [batchDurationMs=${batchDurationMs}ms]"

override lazy val inputRDD: RDD[InternalRow] = {

val inputRDD = new DataSourceRDD(
sparkContext,
partitions,
LowLatencyReaderFactoryWrap(
readerFactory,
LowLatencyClock.getTimeMillis() + batchDurationMs,
endOffsetsAccumulator
),
supportsColumnar,
customMetrics
)
postDriverMetrics()
inputRDD
}
}
Loading