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
9 changes: 9 additions & 0 deletions core/src/main/scala/org/apache/spark/util/RpcUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,15 @@ private[spark] object RpcUtils {
rpcEnv.setupEndpointRef(RpcAddress(driverHost, driverPort), name)
}

def makeDriverRef(
name: String,
driverHost: String,
driverPort: Int,
rpcEnv: RpcEnv): RpcEndpointRef = {
Utils.checkHost(driverHost)
rpcEnv.setupEndpointRef(RpcAddress(driverHost, driverPort), name)
}

/** Returns the default Spark timeout to use for RPC ask operations. */
def askRpcTimeout(conf: SparkConf): RpcTimeout = {
RpcTimeout(conf, Seq(RPC_ASK_TIMEOUT.key, NETWORK_TIMEOUT.key), "120s")
Expand Down
Original file line number Diff line number Diff line change
@@ -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() {}
}
Original file line number Diff line number Diff line change
@@ -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;
}
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
@@ -0,0 +1,191 @@
/*
* 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.execution.datasources.v2

import java.util.Objects

import scala.jdk.OptionConverters._

import org.apache.spark.{SparkContext, TaskContext}
import org.apache.spark.internal.{Logging, LogKeys, MDC}
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.catalyst.plans.logical.Statistics
import org.apache.spark.sql.connector.read.{
InputPartition,
PartitionReader,
PartitionReaderFactory,
Scan,
SupportsReportStatistics
}
import org.apache.spark.sql.connector.read.streaming.{
MicroBatchStream,
Offset,
PartitionOffset,
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, ManualClock, SystemClock}
import org.apache.spark.util.ArrayImplicits._

/* The singleton object to control the time in testing */
object LowLatencyClock {
private var clock: Clock = new SystemClock

def getClock: Clock = clock

def getTimeMillis(): Long = {
clock.getTimeMillis()
}

def waitTillTime(targetTime: Long): Unit = {
clock.waitTillTime(targetTime)
}

/* Below methods are only for testing. */
def setClock(inputClock: Clock): Unit = {
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.
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
}

// There is a rule for the case of TransformWithState + Initial state in realtime mode
// that we overwrite the batch duration to 0 for the first batch. We include
// batchDurationMs in the equals/hashCode methods for the rule to take effect, since
// rule executor will determine the effectiveness of the rule through fast equal.
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] = {
// For RTM task monitoring
sparkContext.setLocalProperty("rtmBatchDurationMs", batchDurationMs.toString)

val inputRDD = new DataSourceRDD(
sparkContext,
partitions,
LowLatencyReaderFactoryWrap(
readerFactory,
LowLatencyClock.getTimeMillis() + batchDurationMs,
endOffsetsAccumulator
),
supportsColumnar,
customMetrics
)
postDriverMetrics()
inputRDD
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -158,6 +158,20 @@ case class MemoryStream[A : Encoder](
id: Int,
sqlContext: SQLContext,
numPartitions: Option[Int] = None)
extends MemoryStreamBaseClass[A](
id, sqlContext, numPartitions = numPartitions)

/**
* A [[Source]] that produces value stored in memory as they are added by the user. This [[Source]]
* is intended for use in unit tests as it can only replay data when the object is still
* available.
*
* If numPartitions is provided, the rows will be redistributed to the given number of partitions.
*/
abstract class MemoryStreamBaseClass[A: Encoder](
id: Int,
sqlContext: SQLContext,
numPartitions: Option[Int] = None)
extends MemoryStreamBase[A](sqlContext)
with MicroBatchStream
with SupportsTriggerAvailableNow
Expand Down Expand Up @@ -298,7 +312,6 @@ case class MemoryStream[A : Encoder](
}
}


class MemoryStreamInputPartition(val records: Array[UnsafeRow]) extends InputPartition

object MemoryStreamReaderFactory extends PartitionReaderFactory {
Expand Down
Loading