Skip to content

Commit 66bd772

Browse files
zsxwingwangzejie
authored andcommitted
[SPARK-20979][SS] Add RateSource to generate values for tests and benchmark
## What changes were proposed in this pull request? This PR adds RateSource for Structured Streaming so that the user can use it to generate data for tests and benchmark easily. This source 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. It supports the following options: - `rowsPerSecond` (e.g. 100, default: 1): How many rows should be generated per second. - `rampUpTime` (e.g. 5s, default: 0s): How long to ramp up before the generating speed becomes `rowsPerSecond`. Using finer granularities than seconds will be truncated to integer seconds. - `numPartitions` (e.g. 10, default: Spark's default parallelism): The partition number for the generated rows. The source will try its best to reach `rowsPerSecond`, but the query may be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed. Here is a simple example that prints 10 rows per seconds: ``` spark.readStream .format("rate") .option("rowsPerSecond", "10") .load() .writeStream .format("console") .start() ``` The idea came from marmbrus and he did the initial work. ## How was this patch tested? The added tests. Author: Shixiong Zhu <[email protected]> Author: Michael Armbrust <[email protected]> Closes apache#18199 from zsxwing/rate.
1 parent 3cd45c8 commit 66bd772

File tree

4 files changed

+429
-0
lines changed

4 files changed

+429
-0
lines changed

sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,3 +5,4 @@ org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
55
org.apache.spark.sql.execution.datasources.text.TextFileFormat
66
org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
77
org.apache.spark.sql.execution.streaming.TextSocketSourceProvider
8+
org.apache.spark.sql.execution.streaming.RateSourceProvider
Lines changed: 243 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,243 @@
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
19+
20+
import java.io._
21+
import java.nio.charset.StandardCharsets
22+
import java.util.concurrent.TimeUnit
23+
24+
import org.apache.commons.io.IOUtils
25+
26+
import org.apache.spark.internal.Logging
27+
import org.apache.spark.network.util.JavaUtils
28+
import org.apache.spark.sql.{DataFrame, SQLContext}
29+
import org.apache.spark.sql.catalyst.InternalRow
30+
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
31+
import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider}
32+
import org.apache.spark.sql.types._
33+
import org.apache.spark.util.{ManualClock, SystemClock}
34+
35+
/**
36+
* A source that generates increment long values with timestamps. Each generated row has two
37+
* columns: a timestamp column for the generated time and an auto increment long column starting
38+
* with 0L.
39+
*
40+
* This source supports the following options:
41+
* - `rowsPerSecond` (e.g. 100, default: 1): How many rows should be generated per second.
42+
* - `rampUpTime` (e.g. 5s, default: 0s): How long to ramp up before the generating speed
43+
* becomes `rowsPerSecond`. Using finer granularities than seconds will be truncated to integer
44+
* seconds.
45+
* - `numPartitions` (e.g. 10, default: Spark's default parallelism): The partition number for the
46+
* generated rows. The source will try its best to reach `rowsPerSecond`, but the query may
47+
* be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed.
48+
*/
49+
class RateSourceProvider extends StreamSourceProvider with DataSourceRegister {
50+
51+
override def sourceSchema(
52+
sqlContext: SQLContext,
53+
schema: Option[StructType],
54+
providerName: String,
55+
parameters: Map[String, String]): (String, StructType) =
56+
(shortName(), RateSourceProvider.SCHEMA)
57+
58+
override def createSource(
59+
sqlContext: SQLContext,
60+
metadataPath: String,
61+
schema: Option[StructType],
62+
providerName: String,
63+
parameters: Map[String, String]): Source = {
64+
val params = CaseInsensitiveMap(parameters)
65+
66+
val rowsPerSecond = params.get("rowsPerSecond").map(_.toLong).getOrElse(1L)
67+
if (rowsPerSecond <= 0) {
68+
throw new IllegalArgumentException(
69+
s"Invalid value '${params("rowsPerSecond")}'. The option 'rowsPerSecond' " +
70+
"must be positive")
71+
}
72+
73+
val rampUpTimeSeconds =
74+
params.get("rampUpTime").map(JavaUtils.timeStringAsSec(_)).getOrElse(0L)
75+
if (rampUpTimeSeconds < 0) {
76+
throw new IllegalArgumentException(
77+
s"Invalid value '${params("rampUpTime")}'. The option 'rampUpTime' " +
78+
"must not be negative")
79+
}
80+
81+
val numPartitions = params.get("numPartitions").map(_.toInt).getOrElse(
82+
sqlContext.sparkContext.defaultParallelism)
83+
if (numPartitions <= 0) {
84+
throw new IllegalArgumentException(
85+
s"Invalid value '${params("numPartitions")}'. The option 'numPartitions' " +
86+
"must be positive")
87+
}
88+
89+
new RateStreamSource(
90+
sqlContext,
91+
metadataPath,
92+
rowsPerSecond,
93+
rampUpTimeSeconds,
94+
numPartitions,
95+
params.get("useManualClock").map(_.toBoolean).getOrElse(false) // Only for testing
96+
)
97+
}
98+
override def shortName(): String = "rate"
99+
}
100+
101+
object RateSourceProvider {
102+
val SCHEMA =
103+
StructType(StructField("timestamp", TimestampType) :: StructField("value", LongType) :: Nil)
104+
105+
val VERSION = 1
106+
}
107+
108+
class RateStreamSource(
109+
sqlContext: SQLContext,
110+
metadataPath: String,
111+
rowsPerSecond: Long,
112+
rampUpTimeSeconds: Long,
113+
numPartitions: Int,
114+
useManualClock: Boolean) extends Source with Logging {
115+
116+
import RateSourceProvider._
117+
import RateStreamSource._
118+
119+
val clock = if (useManualClock) new ManualClock else new SystemClock
120+
121+
private val maxSeconds = Long.MaxValue / rowsPerSecond
122+
123+
if (rampUpTimeSeconds > maxSeconds) {
124+
throw new ArithmeticException(
125+
s"Integer overflow. Max offset with $rowsPerSecond rowsPerSecond" +
126+
s" is $maxSeconds, but 'rampUpTimeSeconds' is $rampUpTimeSeconds.")
127+
}
128+
129+
private val startTimeMs = {
130+
val metadataLog =
131+
new HDFSMetadataLog[LongOffset](sqlContext.sparkSession, metadataPath) {
132+
override def serialize(metadata: LongOffset, out: OutputStream): Unit = {
133+
val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))
134+
writer.write("v" + VERSION + "\n")
135+
writer.write(metadata.json)
136+
writer.flush
137+
}
138+
139+
override def deserialize(in: InputStream): LongOffset = {
140+
val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8))
141+
// HDFSMetadataLog guarantees that it never creates a partial file.
142+
assert(content.length != 0)
143+
if (content(0) == 'v') {
144+
val indexOfNewLine = content.indexOf("\n")
145+
if (indexOfNewLine > 0) {
146+
val version = parseVersion(content.substring(0, indexOfNewLine), VERSION)
147+
LongOffset(SerializedOffset(content.substring(indexOfNewLine + 1)))
148+
} else {
149+
throw new IllegalStateException(
150+
s"Log file was malformed: failed to detect the log file version line.")
151+
}
152+
} else {
153+
throw new IllegalStateException(
154+
s"Log file was malformed: failed to detect the log file version line.")
155+
}
156+
}
157+
}
158+
159+
metadataLog.get(0).getOrElse {
160+
val offset = LongOffset(clock.getTimeMillis())
161+
metadataLog.add(0, offset)
162+
logInfo(s"Start time: $offset")
163+
offset
164+
}.offset
165+
}
166+
167+
/** When the system time runs backward, "lastTimeMs" will make sure we are still monotonic. */
168+
@volatile private var lastTimeMs = startTimeMs
169+
170+
override def schema: StructType = RateSourceProvider.SCHEMA
171+
172+
override def getOffset: Option[Offset] = {
173+
val now = clock.getTimeMillis()
174+
if (lastTimeMs < now) {
175+
lastTimeMs = now
176+
}
177+
Some(LongOffset(TimeUnit.MILLISECONDS.toSeconds(lastTimeMs - startTimeMs)))
178+
}
179+
180+
override def getBatch(start: Option[Offset], end: Offset): DataFrame = {
181+
val startSeconds = start.flatMap(LongOffset.convert(_).map(_.offset)).getOrElse(0L)
182+
val endSeconds = LongOffset.convert(end).map(_.offset).getOrElse(0L)
183+
assert(startSeconds <= endSeconds, s"startSeconds($startSeconds) > endSeconds($endSeconds)")
184+
if (endSeconds > maxSeconds) {
185+
throw new ArithmeticException("Integer overflow. Max offset with " +
186+
s"$rowsPerSecond rowsPerSecond is $maxSeconds, but it's $endSeconds now.")
187+
}
188+
// Fix "lastTimeMs" for recovery
189+
if (lastTimeMs < TimeUnit.SECONDS.toMillis(endSeconds) + startTimeMs) {
190+
lastTimeMs = TimeUnit.SECONDS.toMillis(endSeconds) + startTimeMs
191+
}
192+
val rangeStart = valueAtSecond(startSeconds, rowsPerSecond, rampUpTimeSeconds)
193+
val rangeEnd = valueAtSecond(endSeconds, rowsPerSecond, rampUpTimeSeconds)
194+
logDebug(s"startSeconds: $startSeconds, endSeconds: $endSeconds, " +
195+
s"rangeStart: $rangeStart, rangeEnd: $rangeEnd")
196+
197+
if (rangeStart == rangeEnd) {
198+
return sqlContext.internalCreateDataFrame(sqlContext.sparkContext.emptyRDD, schema)
199+
}
200+
201+
val localStartTimeMs = startTimeMs + TimeUnit.SECONDS.toMillis(startSeconds)
202+
val relativeMsPerValue =
203+
TimeUnit.SECONDS.toMillis(endSeconds - startSeconds).toDouble / (rangeEnd - rangeStart)
204+
205+
val rdd = sqlContext.sparkContext.range(rangeStart, rangeEnd, 1, numPartitions).map { v =>
206+
val relative = math.round((v - rangeStart) * relativeMsPerValue)
207+
InternalRow(DateTimeUtils.fromMillis(relative + localStartTimeMs), v)
208+
}
209+
sqlContext.internalCreateDataFrame(rdd, schema)
210+
}
211+
212+
override def stop(): Unit = {}
213+
214+
override def toString: String = s"RateSource[rowsPerSecond=$rowsPerSecond, " +
215+
s"rampUpTimeSeconds=$rampUpTimeSeconds, numPartitions=$numPartitions]"
216+
}
217+
218+
object RateStreamSource {
219+
220+
/** Calculate the end value we will emit at the time `seconds`. */
221+
def valueAtSecond(seconds: Long, rowsPerSecond: Long, rampUpTimeSeconds: Long): Long = {
222+
// E.g., rampUpTimeSeconds = 4, rowsPerSecond = 10
223+
// Then speedDeltaPerSecond = 2
224+
//
225+
// seconds = 0 1 2 3 4 5 6
226+
// speed = 0 2 4 6 8 10 10 (speedDeltaPerSecond * seconds)
227+
// end value = 0 2 6 12 20 30 40 (0 + speedDeltaPerSecond * seconds) * (seconds + 1) / 2
228+
val speedDeltaPerSecond = rowsPerSecond / (rampUpTimeSeconds + 1)
229+
if (seconds <= rampUpTimeSeconds) {
230+
// Calculate "(0 + speedDeltaPerSecond * seconds) * (seconds + 1) / 2" in a special way to
231+
// avoid overflow
232+
if (seconds % 2 == 1) {
233+
(seconds + 1) / 2 * speedDeltaPerSecond * seconds
234+
} else {
235+
seconds / 2 * speedDeltaPerSecond * (seconds + 1)
236+
}
237+
} else {
238+
// rampUpPart is just a special case of the above formula: rampUpTimeSeconds == seconds
239+
val rampUpPart = valueAtSecond(rampUpTimeSeconds, rowsPerSecond, rampUpTimeSeconds)
240+
rampUpPart + (seconds - rampUpTimeSeconds) * rowsPerSecond
241+
}
242+
}
243+
}

0 commit comments

Comments
 (0)