|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.execution.streaming.sources |
| 19 | + |
| 20 | +import java.{util => ju} |
| 21 | +import java.util.Optional |
| 22 | +import java.util.concurrent.atomic.AtomicInteger |
| 23 | +import javax.annotation.concurrent.GuardedBy |
| 24 | + |
| 25 | +import scala.collection.JavaConverters._ |
| 26 | +import scala.collection.mutable.ListBuffer |
| 27 | + |
| 28 | +import org.json4s.NoTypeHints |
| 29 | +import org.json4s.jackson.Serialization |
| 30 | + |
| 31 | +import org.apache.spark.SparkEnv |
| 32 | +import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} |
| 33 | +import org.apache.spark.sql.{Encoder, Row, SQLContext} |
| 34 | +import org.apache.spark.sql.execution.streaming._ |
| 35 | +import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream.GetRecord |
| 36 | +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions} |
| 37 | +import org.apache.spark.sql.sources.v2.reader.DataReaderFactory |
| 38 | +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousDataReader, ContinuousReader, Offset, PartitionOffset} |
| 39 | +import org.apache.spark.sql.types.StructType |
| 40 | +import org.apache.spark.util.RpcUtils |
| 41 | + |
| 42 | +/** |
| 43 | + * The overall strategy here is: |
| 44 | + * * ContinuousMemoryStream maintains a list of records for each partition. addData() will |
| 45 | + * distribute records evenly-ish across partitions. |
| 46 | + * * RecordEndpoint is set up as an endpoint for executor-side |
| 47 | + * ContinuousMemoryStreamDataReader instances to poll. It returns the record at the specified |
| 48 | + * offset within the list, or null if that offset doesn't yet have a record. |
| 49 | + */ |
| 50 | +class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) |
| 51 | + extends MemoryStreamBase[A](sqlContext) with ContinuousReader with ContinuousReadSupport { |
| 52 | + private implicit val formats = Serialization.formats(NoTypeHints) |
| 53 | + private val NUM_PARTITIONS = 2 |
| 54 | + |
| 55 | + protected val logicalPlan = |
| 56 | + StreamingRelationV2(this, "memory", Map(), attributes, None)(sqlContext.sparkSession) |
| 57 | + |
| 58 | + // ContinuousReader implementation |
| 59 | + |
| 60 | + @GuardedBy("this") |
| 61 | + private val records = Seq.fill(NUM_PARTITIONS)(new ListBuffer[A]) |
| 62 | + |
| 63 | + @GuardedBy("this") |
| 64 | + private var startOffset: ContinuousMemoryStreamOffset = _ |
| 65 | + |
| 66 | + private val recordEndpoint = new RecordEndpoint() |
| 67 | + @volatile private var endpointRef: RpcEndpointRef = _ |
| 68 | + |
| 69 | + def addData(data: TraversableOnce[A]): Offset = synchronized { |
| 70 | + // Distribute data evenly among partition lists. |
| 71 | + data.toSeq.zipWithIndex.map { |
| 72 | + case (item, index) => records(index % NUM_PARTITIONS) += item |
| 73 | + } |
| 74 | + |
| 75 | + // The new target offset is the offset where all records in all partitions have been processed. |
| 76 | + ContinuousMemoryStreamOffset((0 until NUM_PARTITIONS).map(i => (i, records(i).size)).toMap) |
| 77 | + } |
| 78 | + |
| 79 | + override def setStartOffset(start: Optional[Offset]): Unit = synchronized { |
| 80 | + // Inferred initial offset is position 0 in each partition. |
| 81 | + startOffset = start.orElse { |
| 82 | + ContinuousMemoryStreamOffset((0 until NUM_PARTITIONS).map(i => (i, 0)).toMap) |
| 83 | + }.asInstanceOf[ContinuousMemoryStreamOffset] |
| 84 | + } |
| 85 | + |
| 86 | + override def getStartOffset: Offset = synchronized { |
| 87 | + startOffset |
| 88 | + } |
| 89 | + |
| 90 | + override def deserializeOffset(json: String): ContinuousMemoryStreamOffset = { |
| 91 | + ContinuousMemoryStreamOffset(Serialization.read[Map[Int, Int]](json)) |
| 92 | + } |
| 93 | + |
| 94 | + override def mergeOffsets(offsets: Array[PartitionOffset]): ContinuousMemoryStreamOffset = { |
| 95 | + ContinuousMemoryStreamOffset( |
| 96 | + offsets.map { |
| 97 | + case ContinuousMemoryStreamPartitionOffset(part, num) => (part, num) |
| 98 | + }.toMap |
| 99 | + ) |
| 100 | + } |
| 101 | + |
| 102 | + override def createDataReaderFactories(): ju.List[DataReaderFactory[Row]] = { |
| 103 | + synchronized { |
| 104 | + val endpointName = s"ContinuousMemoryStreamRecordEndpoint-${java.util.UUID.randomUUID()}-$id" |
| 105 | + endpointRef = |
| 106 | + recordEndpoint.rpcEnv.setupEndpoint(endpointName, recordEndpoint) |
| 107 | + |
| 108 | + startOffset.partitionNums.map { |
| 109 | + case (part, index) => |
| 110 | + new ContinuousMemoryStreamDataReaderFactory( |
| 111 | + endpointName, part, index): DataReaderFactory[Row] |
| 112 | + }.toList.asJava |
| 113 | + } |
| 114 | + } |
| 115 | + |
| 116 | + override def stop(): Unit = { |
| 117 | + if (endpointRef != null) recordEndpoint.rpcEnv.stop(endpointRef) |
| 118 | + } |
| 119 | + |
| 120 | + override def commit(end: Offset): Unit = {} |
| 121 | + |
| 122 | + // ContinuousReadSupport implementation |
| 123 | + // This is necessary because of how StreamTest finds the source for AddDataMemory steps. |
| 124 | + def createContinuousReader( |
| 125 | + schema: Optional[StructType], |
| 126 | + checkpointLocation: String, |
| 127 | + options: DataSourceOptions): ContinuousReader = { |
| 128 | + this |
| 129 | + } |
| 130 | + |
| 131 | + /** |
| 132 | + * Endpoint for executors to poll for records. |
| 133 | + */ |
| 134 | + private class RecordEndpoint extends ThreadSafeRpcEndpoint { |
| 135 | + override val rpcEnv: RpcEnv = SparkEnv.get.rpcEnv |
| 136 | + |
| 137 | + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { |
| 138 | + case GetRecord(ContinuousMemoryStreamPartitionOffset(part, index)) => |
| 139 | + ContinuousMemoryStream.this.synchronized { |
| 140 | + val buf = records(part) |
| 141 | + val record = if (buf.size <= index) None else Some(buf(index)) |
| 142 | + |
| 143 | + context.reply(record.map(Row(_))) |
| 144 | + } |
| 145 | + } |
| 146 | + } |
| 147 | +} |
| 148 | + |
| 149 | +object ContinuousMemoryStream { |
| 150 | + case class GetRecord(offset: ContinuousMemoryStreamPartitionOffset) |
| 151 | + protected val memoryStreamId = new AtomicInteger(0) |
| 152 | + |
| 153 | + def apply[A : Encoder](implicit sqlContext: SQLContext): ContinuousMemoryStream[A] = |
| 154 | + new ContinuousMemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext) |
| 155 | +} |
| 156 | + |
| 157 | +/** |
| 158 | + * Data reader factory for continuous memory stream. |
| 159 | + */ |
| 160 | +class ContinuousMemoryStreamDataReaderFactory( |
| 161 | + driverEndpointName: String, |
| 162 | + partition: Int, |
| 163 | + startOffset: Int) extends DataReaderFactory[Row] { |
| 164 | + override def createDataReader: ContinuousMemoryStreamDataReader = |
| 165 | + new ContinuousMemoryStreamDataReader(driverEndpointName, partition, startOffset) |
| 166 | +} |
| 167 | + |
| 168 | +/** |
| 169 | + * Data reader for continuous memory stream. |
| 170 | + * |
| 171 | + * Polls the driver endpoint for new records. |
| 172 | + */ |
| 173 | +class ContinuousMemoryStreamDataReader( |
| 174 | + driverEndpointName: String, |
| 175 | + partition: Int, |
| 176 | + startOffset: Int) extends ContinuousDataReader[Row] { |
| 177 | + private val endpoint = RpcUtils.makeDriverRef( |
| 178 | + driverEndpointName, |
| 179 | + SparkEnv.get.conf, |
| 180 | + SparkEnv.get.rpcEnv) |
| 181 | + |
| 182 | + private var currentOffset = startOffset |
| 183 | + private var current: Option[Row] = None |
| 184 | + |
| 185 | + override def next(): Boolean = { |
| 186 | + current = None |
| 187 | + while (current.isEmpty) { |
| 188 | + Thread.sleep(10) |
| 189 | + current = endpoint.askSync[Option[Row]]( |
| 190 | + GetRecord(ContinuousMemoryStreamPartitionOffset(partition, currentOffset))) |
| 191 | + } |
| 192 | + currentOffset += 1 |
| 193 | + true |
| 194 | + } |
| 195 | + |
| 196 | + override def get(): Row = current.get |
| 197 | + |
| 198 | + override def close(): Unit = {} |
| 199 | + |
| 200 | + override def getOffset: ContinuousMemoryStreamPartitionOffset = |
| 201 | + ContinuousMemoryStreamPartitionOffset(partition, currentOffset) |
| 202 | +} |
| 203 | + |
| 204 | +case class ContinuousMemoryStreamOffset(partitionNums: Map[Int, Int]) |
| 205 | + extends Offset { |
| 206 | + private implicit val formats = Serialization.formats(NoTypeHints) |
| 207 | + override def json(): String = Serialization.write(partitionNums) |
| 208 | +} |
| 209 | + |
| 210 | +case class ContinuousMemoryStreamPartitionOffset(partition: Int, numProcessed: Int) |
| 211 | + extends PartitionOffset |
0 commit comments