-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-20979][SS]Add RateSource to generate values for tests and benchmark #18199
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
554eafb
5b45b1b
3a95b55
479582b
ad32a7f
240c27b
f471651
d5e7492
1d8454d
a2fa0b7
53a65fb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,208 @@ | ||
| /* | ||
| * 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.io._ | ||
| import java.nio.charset.StandardCharsets | ||
| import java.util.concurrent.TimeUnit | ||
|
|
||
| import org.apache.commons.io.IOUtils | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.{DataFrame, SQLContext} | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} | ||
| import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} | ||
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.util.{ManualClock, SystemClock} | ||
|
|
||
| /** | ||
| * A source that generates increment long values with timestamps. Each generated row has two | ||
| * columns: a timestamp column for the generated time and an auto increment long column starting | ||
| * with 0L. | ||
| * | ||
| * This source supports the following options: | ||
| * - `tuplesPerSecond` (default: 1): How many tuples should be generated per second. | ||
| * - `rampUpTimeSeconds` (default: 0): How many seconds to ramp up before the generating speed | ||
| * becomes `tuplesPerSecond`. | ||
| * - `numPartitions` (default: Spark's default parallelism): The partition number for the generated | ||
| * tuples. | ||
| */ | ||
| class RateSourceProvider extends StreamSourceProvider with DataSourceRegister { | ||
|
|
||
| override def sourceSchema( | ||
| sqlContext: SQLContext, | ||
| schema: Option[StructType], | ||
| providerName: String, | ||
| parameters: Map[String, String]): (String, StructType) = | ||
| (shortName(), RateSourceProvider.SCHEMA) | ||
|
|
||
| override def createSource( | ||
| sqlContext: SQLContext, | ||
| metadataPath: String, | ||
| schema: Option[StructType], | ||
| providerName: String, | ||
| parameters: Map[String, String]): Source = { | ||
| val params = CaseInsensitiveMap(parameters) | ||
|
|
||
| val tuplesPerSecond = params.get("tuplesPerSecond").map(_.toLong).getOrElse(1L) | ||
| if (tuplesPerSecond <= 0) { | ||
| throw new IllegalArgumentException( | ||
| s"Invalid value '${params("tuplesPerSecond")}' for option 'tuplesPerSecond', " + | ||
| "must be positive") | ||
| } | ||
|
|
||
| val rampUpTimeSeconds = params.get("rampUpTimeSeconds").map(_.toLong).getOrElse(0L) | ||
|
||
| if (rampUpTimeSeconds < 0) { | ||
| throw new IllegalArgumentException( | ||
| s"Invalid value '${params("rampUpTimeSeconds")}' for option 'rampUpTimeSeconds', " + | ||
| "must not be negative") | ||
| } | ||
|
|
||
| val numPartitions = params.get("numPartitions").map(_.toInt).getOrElse( | ||
| sqlContext.sparkContext.defaultParallelism) | ||
| if (numPartitions <= 0) { | ||
| throw new IllegalArgumentException( | ||
| s"Invalid value '${params("numPartitions")}' for option 'numPartitions', " + | ||
| "must be positive") | ||
| } | ||
|
|
||
| new RateStreamSource( | ||
| sqlContext, | ||
| metadataPath, | ||
| tuplesPerSecond, | ||
| rampUpTimeSeconds, | ||
| numPartitions, | ||
| params.get("useManualClock").map(_.toBoolean).getOrElse(false) // Only for testing | ||
| ) | ||
| } | ||
| override def shortName(): String = "rate" | ||
| } | ||
|
|
||
| object RateSourceProvider { | ||
| val SCHEMA = | ||
| StructType(StructField("timestamp", TimestampType) :: StructField("value", LongType) :: Nil) | ||
|
|
||
| val VERSION = 1 | ||
| } | ||
|
|
||
| class RateStreamSource( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should we add a
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think so. The class won't appear in the public Scaladoc/Javadoc. The user cannot see this tag in any place unless they jump to this file. |
||
| sqlContext: SQLContext, | ||
| metadataPath: String, | ||
| tuplesPerSecond: Long, | ||
| rampUpTimeSeconds: Long, | ||
| numPartitions: Int, | ||
| useManualClock: Boolean) extends Source with Logging { | ||
|
|
||
| import RateSourceProvider._ | ||
|
|
||
| val clock = if (useManualClock) new ManualClock else new SystemClock | ||
|
|
||
| private val maxSeconds = Long.MaxValue / tuplesPerSecond | ||
|
||
|
|
||
| if (rampUpTimeSeconds > maxSeconds) { | ||
| throw new ArithmeticException("integer overflow. Max offset with tuplesPerSecond " + | ||
|
||
| s"$tuplesPerSecond is $maxSeconds, but 'rampUpTimeSeconds' is $rampUpTimeSeconds.") | ||
| } | ||
|
|
||
| private val startTimeMs = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we need to go to this complexity for this source??
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's better to add versioning at the beginning. Just a lesson from Kafka source. |
||
| val metadataLog = | ||
| new HDFSMetadataLog[LongOffset](sqlContext.sparkSession, metadataPath) { | ||
| override def serialize(metadata: LongOffset, out: OutputStream): Unit = { | ||
| val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)) | ||
| writer.write("v" + VERSION + "\n") | ||
| writer.write(metadata.json) | ||
| writer.flush | ||
| } | ||
|
|
||
| override def deserialize(in: InputStream): LongOffset = { | ||
| val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8)) | ||
| // HDFSMetadataLog guarantees that it never creates a partial file. | ||
| assert(content.length != 0) | ||
| if (content(0) == 'v') { | ||
| val indexOfNewLine = content.indexOf("\n") | ||
| if (indexOfNewLine > 0) { | ||
| val version = parseVersion(content.substring(0, indexOfNewLine), VERSION) | ||
| LongOffset(SerializedOffset(content.substring(indexOfNewLine + 1))) | ||
| } else { | ||
| throw new IllegalStateException( | ||
| s"Log file was malformed: failed to detect the log file version line.") | ||
| } | ||
| } else { | ||
| throw new IllegalStateException( | ||
| s"Log file was malformed: failed to detect the log file version line.") | ||
| } | ||
| } | ||
| } | ||
|
|
||
| metadataLog.get(0).getOrElse { | ||
| val offset = LongOffset(clock.getTimeMillis()) | ||
| metadataLog.add(0, offset) | ||
| logInfo(s"Start time: $offset") | ||
| offset | ||
| }.offset | ||
| } | ||
|
|
||
| /** When the system time runs backward, "lastTimeMs" will make sure we are still monotonic. */ | ||
| @volatile private var lastTimeMs = startTimeMs | ||
|
|
||
| override def schema: StructType = RateSourceProvider.SCHEMA | ||
|
|
||
| override def getOffset: Option[Offset] = { | ||
| val now = clock.getTimeMillis() | ||
| if (lastTimeMs < now) { | ||
| lastTimeMs = now | ||
| } | ||
| Some(LongOffset(TimeUnit.MILLISECONDS.toSeconds(lastTimeMs - startTimeMs))) | ||
| } | ||
|
|
||
| override def getBatch(start: Option[Offset], end: Offset): DataFrame = { | ||
| val startSeconds = start.flatMap(LongOffset.convert(_).map(_.offset)).getOrElse(0L) | ||
| val endSeconds = LongOffset.convert(end).map(_.offset).getOrElse(0L) | ||
| assert(startSeconds <= endSeconds) | ||
|
||
| if (endSeconds > maxSeconds) { | ||
| throw new ArithmeticException("integer overflow. Max offset with " + | ||
|
||
| s"tuplesPerSecond $tuplesPerSecond is $maxSeconds, but it's $endSeconds now.") | ||
| } | ||
| // Fix "lastTimeMs" for recovery | ||
| if (lastTimeMs < TimeUnit.SECONDS.toMillis(endSeconds) + startTimeMs) { | ||
| lastTimeMs = TimeUnit.SECONDS.toMillis(endSeconds) + startTimeMs | ||
| } | ||
| val (rangeStart, rangeEnd) = if (rampUpTimeSeconds > endSeconds) { | ||
| (math.rint(tuplesPerSecond * (startSeconds * 1.0 / rampUpTimeSeconds)).toLong * startSeconds, | ||
| math.rint(tuplesPerSecond * (endSeconds * 1.0 / rampUpTimeSeconds)).toLong * endSeconds) | ||
| } else if (startSeconds < rampUpTimeSeconds) { | ||
| (math.rint(tuplesPerSecond * (startSeconds * 1.0 / rampUpTimeSeconds)).toLong * startSeconds, | ||
| endSeconds * tuplesPerSecond) | ||
| } else { | ||
| (startSeconds * tuplesPerSecond, endSeconds * tuplesPerSecond) | ||
| } | ||
| logDebug(s"startSeconds: $startSeconds, endSeconds: $endSeconds, " + | ||
| s"rangeStart: $rangeStart, rangeEnd: $rangeEnd") | ||
| val localStartTimeMs = startTimeMs | ||
| val localPerSecond = tuplesPerSecond | ||
|
|
||
| val rdd = sqlContext.sparkContext.range(rangeStart, rangeEnd, 1, numPartitions).map { v => | ||
| val relative = v * 1000L / localPerSecond | ||
| InternalRow(DateTimeUtils.fromMillis(relative + localStartTimeMs), v) | ||
| } | ||
| sqlContext.internalCreateDataFrame(rdd, schema) | ||
| } | ||
|
|
||
| override def stop(): Unit = {} | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,148 @@ | ||
| /* | ||
| * 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.TimeUnit | ||
|
|
||
| import org.apache.spark.sql.functions._ | ||
| import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest} | ||
| import org.apache.spark.util.ManualClock | ||
|
|
||
| class RateSourceSuite extends StreamTest { | ||
|
|
||
| import testImplicits._ | ||
|
|
||
| case class AdvanceRateManualClock(seconds: Long) extends AddData { | ||
| override def addData(query: Option[StreamExecution]): (Source, Offset) = { | ||
| assert(query.nonEmpty) | ||
| val rateSource = query.get.logicalPlan.collect { | ||
| case StreamingExecutionRelation(source, _) if source.isInstanceOf[RateStreamSource] => | ||
| source.asInstanceOf[RateStreamSource] | ||
| }.head | ||
| rateSource.clock.asInstanceOf[ManualClock].advance(TimeUnit.SECONDS.toMillis(seconds)) | ||
| (rateSource, rateSource.getOffset.get) | ||
| } | ||
| } | ||
|
|
||
| private def getManualClockFromQuery(query: StreamExecution): ManualClock = { | ||
| val rateSource = query.logicalPlan.collect { | ||
| case StreamingExecutionRelation(source, _) if source.isInstanceOf[RateStreamSource] => | ||
| source.asInstanceOf[RateStreamSource] | ||
| }.head | ||
| rateSource.clock.asInstanceOf[ManualClock] | ||
| } | ||
|
|
||
| test("basic") { | ||
| val input = spark.readStream | ||
| .format("rate") | ||
| .option("tuplesPerSecond", "10") | ||
| .option("useManualClock", "true") | ||
| .load() | ||
| testStream(input)( | ||
| AdvanceRateManualClock(seconds = 1), | ||
| CheckLastBatch((0 until 10).map(v => new java.sql.Timestamp(v * 100L) -> v): _*), | ||
| StopStream, | ||
| StartStream(), | ||
| // Advance 2 seconds because creating a new RateSource will also create a new ManualClock | ||
| AdvanceRateManualClock(seconds = 2), | ||
| CheckLastBatch((10 until 20).map(v => new java.sql.Timestamp(v * 100L) -> v): _*) | ||
| ) | ||
| } | ||
|
|
||
| test("rampUpTimeSeconds") { | ||
| val input = spark.readStream | ||
| .format("rate") | ||
| .option("tuplesPerSecond", "10") | ||
| .option("rampUpTimeSeconds", "5") | ||
| .option("useManualClock", "true") | ||
| .load() | ||
| .select($"value") | ||
| testStream(input)( | ||
| AdvanceRateManualClock(seconds = 1), | ||
| CheckLastBatch((0 until 2): _*), | ||
| AdvanceRateManualClock(seconds = 1), | ||
| CheckLastBatch((2 until 8): _*), | ||
| AdvanceRateManualClock(seconds = 1), | ||
| CheckLastBatch((8 until 18): _*), | ||
| AdvanceRateManualClock(seconds = 1), | ||
| CheckLastBatch((18 until 32): _*), | ||
| AdvanceRateManualClock(seconds = 1), | ||
| CheckLastBatch((32 until 50): _*), | ||
| AdvanceRateManualClock(seconds = 1), | ||
| CheckLastBatch((50 until 60): _*) | ||
| ) | ||
| } | ||
|
|
||
| test("numPartitions") { | ||
| val input = spark.readStream | ||
| .format("rate") | ||
| .option("tuplesPerSecond", "10") | ||
| .option("numPartitions", "6") | ||
| .option("useManualClock", "true") | ||
| .load() | ||
| .select(spark_partition_id()) | ||
| .distinct() | ||
| testStream(input)( | ||
| AdvanceRateManualClock(1), | ||
| CheckLastBatch((0 until 6): _*) | ||
| ) | ||
| } | ||
|
|
||
| testQuietly("overflow") { | ||
| val input = spark.readStream | ||
| .format("rate") | ||
| .option("tuplesPerSecond", Long.MaxValue.toString) | ||
| .option("useManualClock", "true") | ||
| .load() | ||
| .select(spark_partition_id()) | ||
| .distinct() | ||
| testStream(input)( | ||
| AdvanceRateManualClock(2), | ||
| ExpectFailure[ArithmeticException](t => { | ||
| Seq("overflow", "tuplesPerSecond").foreach { msg => | ||
| assert(t.getMessage.contains(msg)) | ||
| } | ||
| }) | ||
| ) | ||
| } | ||
|
|
||
| testQuietly("illegal option values") { | ||
| def testIllegalOptionValue( | ||
| option: String, | ||
| value: String, | ||
| expectedMessages: Seq[String]): Unit = { | ||
| val e = intercept[StreamingQueryException] { | ||
| spark.readStream | ||
| .format("rate") | ||
| .option(option, value) | ||
| .load() | ||
| .writeStream | ||
| .format("console") | ||
| .start() | ||
| .awaitTermination() | ||
| } | ||
| assert(e.getCause.isInstanceOf[IllegalArgumentException]) | ||
| for (msg <- expectedMessages) { | ||
| assert(e.getCause.getMessage.contains(msg)) | ||
| } | ||
| } | ||
|
|
||
| testIllegalOptionValue("tuplesPerSecond", "-1", Seq("-1", "tuplesPerSecond", "positive")) | ||
| testIllegalOptionValue("numPartitions", "-1", Seq("-1", "numPartitions", "positive")) | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit:
Invalid value '${params("tuplesPerSecond")}'. The option 'tuplesPerSecond' must be a positive?