viirya commented on code in PR #52502: URL: https://github.com/apache/spark/pull/52502#discussion_r2433337678
########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/LowLatencyMemoryStream.scala: ########## @@ -0,0 +1,300 @@ +/* + * 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.streaming + +import java.util.concurrent.atomic.AtomicInteger +import javax.annotation.concurrent.GuardedBy + +import scala.collection.mutable.ListBuffer + +import org.json4s.{Formats, NoTypeHints} +import org.json4s.jackson.Serialization + +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} +import org.apache.spark.sql.{Encoder, SQLContext} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.connector.read.streaming.{ + Offset => OffsetV2, + PartitionOffset, + ReadLimit, + SupportsRealTimeMode, + SupportsRealTimeRead +} +import org.apache.spark.sql.connector.read.streaming.SupportsRealTimeRead.RecordStatus +import org.apache.spark.sql.execution.datasources.v2.LowLatencyClock +import org.apache.spark.sql.execution.streaming.runtime._ +import org.apache.spark.util.{Clock, RpcUtils} + +/** + * A low latency memory source from memory, only for unit test purpose. + * This class is very similar to ContinuousMemoryStream, except that it implements the + * interface of SupportsRealTimeMode, rather than ContinuousStream + * The overall strategy here is: + * * LowLatencyMemoryStream maintains a list of records for each partition. addData() will + * distribute records evenly-ish across partitions. + * * RecordEndpoint is set up as an endpoint for executor-side + * LowLatencyMemoryStreamInputPartitionReader instances to poll. It returns the record at + * the specified offset within the list, or null if that offset doesn't yet have a record. + */ +class LowLatencyMemoryStream[A: Encoder]( + id: Int, + sqlContext: SQLContext, + numPartitions: Int = 2, + clock: Clock = LowLatencyClock.getClock) + extends MemoryStreamBaseClass[A](0, sqlContext) + with SupportsRealTimeMode { + private implicit val formats: Formats = Serialization.formats(NoTypeHints) + + // LowLatencyReader implementation + + @GuardedBy("this") + private val records = Seq.fill(numPartitions)(new ListBuffer[UnsafeRow]) + + private val recordEndpoint = new ContinuousRecordEndpoint(records, this) + @volatile private var endpointRef: RpcEndpointRef = _ + + override def addData(data: IterableOnce[A]): Offset = synchronized { + // Distribute data evenly among partition lists. + data.iterator.to(Seq).zipWithIndex.map { + case (item, index) => + records(index % numPartitions) += toRow(item).copy().asInstanceOf[UnsafeRow] + } + + // The new target offset is the offset where all records in all partitions have been processed. + LowLatencyMemoryStreamOffset((0 until numPartitions).map(i => (i, records(i).size)).toMap) + } + + def addData(partitionId: Int, data: IterableOnce[A]): Offset = synchronized { + require( + partitionId >= 0 && partitionId < numPartitions, + s"Partition ID $partitionId is out of bounds for $numPartitions partitions." + ) + + // Add data to the specified partition. + records(partitionId) ++= data.iterator.map(item => toRow(item).copy().asInstanceOf[UnsafeRow]) + + // The new target offset is the offset where all records in all partitions have been processed. + LowLatencyMemoryStreamOffset((0 until numPartitions).map(i => (i, records(i).size)).toMap) + } + + override def initialOffset(): OffsetV2 = { + LowLatencyMemoryStreamOffset((0 until numPartitions).map(i => (i, 0)).toMap) + } + + override def latestOffset(startOffset: OffsetV2, limit: ReadLimit): OffsetV2 = { + LowLatencyMemoryStreamOffset((0 until numPartitions).map(i => (i, records(i).size)).toMap) + } + + override def deserializeOffset(json: String): LowLatencyMemoryStreamOffset = { + LowLatencyMemoryStreamOffset(Serialization.read[Map[Int, Int]](json)) + } + + override def mergeOffsets(offsets: Array[PartitionOffset]): LowLatencyMemoryStreamOffset = { + LowLatencyMemoryStreamOffset( + offsets.map { + case ContinuousRecordPartitionOffset(part, num) => (part, num) + }.toMap + ) + } + + override def planInputPartitions(start: OffsetV2): Array[InputPartition] = { + val startOffset = start.asInstanceOf[LowLatencyMemoryStreamOffset] + synchronized { + val endpointName = s"ContinuousRecordEndpoint-${java.util.UUID.randomUUID()}-$id" + endpointRef = recordEndpoint.rpcEnv.setupEndpoint(endpointName, recordEndpoint) + + startOffset.partitionNums.map { + case (part, index) => + LowLatencyMemoryStreamInputPartition( + endpointName, + endpointRef.address, + part, + index, + Int.MaxValue + ) + }.toArray + } + } + + override def planInputPartitions(start: OffsetV2, end: OffsetV2): Array[InputPartition] = { + val startOffset = start.asInstanceOf[LowLatencyMemoryStreamOffset] + val endOffset = end.asInstanceOf[LowLatencyMemoryStreamOffset] + synchronized { + val endpointName = s"ContinuousRecordEndpoint-${java.util.UUID.randomUUID()}-$id" + endpointRef = recordEndpoint.rpcEnv.setupEndpoint(endpointName, recordEndpoint) + + startOffset.partitionNums.map { + case (part, index) => + LowLatencyMemoryStreamInputPartition( + endpointName, + endpointRef.address, + part, + index, + endOffset.partitionNums(part) + ) + }.toArray + } + } + + override def createReaderFactory(): PartitionReaderFactory = { + new LowLatencyMemoryStreamReaderFactory(clock) + } + + override def stop(): Unit = { + if (endpointRef != null) recordEndpoint.rpcEnv.stop(endpointRef) + } + + override def commit(end: OffsetV2): Unit = {} + + override def reset(): Unit = { + super.reset() + records.foreach(_.clear()) + } +} + +object LowLatencyMemoryStream { + protected val memoryStreamId = new AtomicInteger(0) + + def apply[A: Encoder](implicit sqlContext: SQLContext): LowLatencyMemoryStream[A] = + new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext) + + def apply[A: Encoder](numPartitions: Int)( + implicit + sqlContext: SQLContext): LowLatencyMemoryStream[A] = + new LowLatencyMemoryStream[A]( + memoryStreamId.getAndIncrement(), + sqlContext, + numPartitions = numPartitions + ) + + def singlePartition[A: Encoder](implicit sqlContext: SQLContext): LowLatencyMemoryStream[A] = + new LowLatencyMemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext, 1) +} + +/** + * An input partition for LowLatency memory stream. + */ +case class LowLatencyMemoryStreamInputPartition( + driverEndpointName: String, + driverEndpointAddress: RpcAddress, + partition: Int, + startOffset: Int, + endOffset: Int) + extends InputPartition + +class LowLatencyMemoryStreamReaderFactory(clock: Clock) extends PartitionReaderFactory { + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + val p = partition.asInstanceOf[LowLatencyMemoryStreamInputPartition] + new LowLatencyMemoryStreamPartitionReader( + p.driverEndpointName, + p.driverEndpointAddress, + p.partition, + p.startOffset, + p.endOffset, + clock + ) + } +} + +/** + * An input partition reader for LowLatency memory stream. + * + * Polls the driver endpoint for new records. + */ +class LowLatencyMemoryStreamPartitionReader( + driverEndpointName: String, + driverEndpointAddress: RpcAddress, + partition: Int, + startOffset: Int, + endOffset: Int, + clock: Clock) + extends SupportsRealTimeRead[InternalRow] { + // ES-1365239: Avoid tracking the ref, given that we create a new one for each partition reader Review Comment: ```suggestion // Avoid tracking the ref, given that we create a new one for each partition reader ``` -- 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
