Skip to content

Commit 01f0ea8

Browse files
committed
[KYUUBI #1743] Fix parallelism of DataGenerator and other enhancements
### _Why are the changes needed?_ The parallelism of DataGenerator always is `spark.sparkContext.defaultParallelism`, it does not make sense for generating large scale data. ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/latest/develop_tools/testing.html#running-tests) locally before make a pull request Closes #1743 from pan3793/tpcds. Closes #1743 62f7c86 [Cheng Pan] nit fdcf832 [Cheng Pan] nit a52ff48 [Cheng Pan] Fix parallelism of DataGenerator and other enhancements Authored-by: Cheng Pan <chengpan@apache.org> Signed-off-by: Cheng Pan <chengpan@apache.org>
1 parent ab4184d commit 01f0ea8

File tree

8 files changed

+97
-98
lines changed

8 files changed

+97
-98
lines changed

dev/kyuubi-tpcds/README.md

Lines changed: 18 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -16,57 +16,60 @@
1616
-->
1717

1818
# Introduction
19-
This module includes tpcds data generator and benchmark.
19+
This module includes TPC-DS data generator and benchmark tool.
2020

2121
# How to use
2222

2323
package jar with following command:
24-
`./build/mvn install -DskipTests -Ptpcds -pl dev/kyuubi-tpcds -am`
24+
`./build/mvn clean package -Ptpcds -pl dev/kyuubi-tpcds -am`
2525

26-
## data generator
26+
## Data Generator
2727

2828
Support options:
2929

30-
| key | default | description |
31-
|-------------|---------|------------------------------|
32-
| db | default | the databases to write data |
33-
| scaleFactor | 1 | the scale factor of tpcds |
30+
| key | default | description |
31+
|--------------|-----------------|-----------------------------------|
32+
| db | default | the database to write data |
33+
| scaleFactor | 1 | the scale factor of TPC-DS |
34+
| format | parquet | the format of table to store data |
35+
| parallel | scaleFactor * 2 | the parallelism of Spark job |
3436

3537
Example: the following command to generate 10GB data with new database `tpcds_sf10`.
3638

3739
```shell
3840
$SPARK_HOME/bin/spark-submit \
3941
--class org.apache.kyuubi.tpcds.DataGenerator \
40-
kyuubi-tpcds-*.jar --db tpcds_sf10 --scaleFactor 10
42+
kyuubi-tpcds_*.jar \
43+
--db tpcds_sf10 --scaleFactor 10 --format parquet --parallel 20
4144
```
4245

43-
## do benchmark
46+
## Benchmark Tool
4447

4548
Support options:
4649

4750
| key | default | description |
4851
|------------|----------------------|--------------------------------------------------------|
49-
| db | none(required) | the tpcds database |
52+
| db | none(required) | the TPC-DS database |
5053
| benchmark | tpcds-v2.4-benchmark | the name of application |
5154
| iterations | 3 | the number of iterations to run |
5255
| filter | a | filter on the name of the queries to run, e.g. q1-v2.4 |
5356

54-
Example: the following command to benchmark tpcds sf10 with exists database `tpcds_sf10`.
57+
Example: the following command to benchmark TPC-DS sf10 with exists database `tpcds_sf10`.
5558

5659
```shell
5760
$SPARK_HOME/bin/spark-submit \
5861
--class org.apache.kyuubi.tpcds.benchmark.RunBenchmark \
59-
kyuubi-tpcds-*.jar --db tpcds_sf10
62+
kyuubi-tpcds_*.jar --db tpcds_sf10
6063
```
6164

62-
We also support run one of the tpcds query:
65+
We also support run one of the TPC-DS query:
6366
```shell
6467
$SPARK_HOME/bin/spark-submit \
6568
--class org.apache.kyuubi.tpcds.benchmark.RunBenchmark \
66-
kyuubi-tpcds-*.jar --db tpcds_sf10 --filter q1-v2.4
69+
kyuubi-tpcds_*.jar --db tpcds_sf10 --filter q1-v2.4
6770
```
6871

69-
The result of tpcds benchmark like:
72+
The result of TPC-DS benchmark like:
7073

7174
| name | minTimeMs | maxTimeMs | avgTimeMs | stdDev | stdDevPercent |
7275
|---------|-----------|-------------|------------|----------|----------------|

dev/kyuubi-tpcds/src/main/scala/org/apache/kyuubi/tpcds/DataGenerator.scala

Lines changed: 34 additions & 39 deletions
Original file line numberDiff line numberDiff line change
@@ -17,32 +17,29 @@
1717

1818
package org.apache.kyuubi.tpcds
1919

20-
import org.apache.spark.SparkConf
2120
import org.apache.spark.sql.SparkSession
2221
import org.slf4j.LoggerFactory
2322

24-
case class RunConfig(
25-
db: String = "default",
26-
scaleFactor: Int = 1)
27-
2823
/**
2924
* Usage:
3025
* <p>
3126
* Run following command to generate 10GB data with new database `tpcds_sf10`.
3227
* {{{
33-
* `$SPARK_HOME/bin/spark-submit \
34-
* --conf spark.sql.tpcds.scale.factor=10 \
35-
* --conf spark.sql.tpcds.database=tpcds_sf10 \
28+
* $SPARK_HOME/bin/spark-submit \
3629
* --class org.apache.kyuubi.tpcds.DataGenerator \
37-
* kyuubi-tpcds-*.jar`
30+
* kyuubi-tpcds_*.jar \
31+
* --db tpcds_sf10 --scaleFactor 10 --format parquet --parallel 20
3832
* }}}
3933
*/
4034
object DataGenerator {
41-
private val logger =
42-
LoggerFactory.getLogger(this.getClass.getSimpleName.stripSuffix("$"))
4335

44-
val SCALE_FACTOR_KEY = "spark.sql.tpcds.scale.factor"
45-
val DATABASE_KEY = "spark.sql.tpcds.database"
36+
case class Config(
37+
db: String = "default",
38+
scaleFactor: Int = 1,
39+
format: String = "parquet",
40+
parallel: Option[Int] = None)
41+
42+
private val logger = LoggerFactory.getLogger(this.getClass.getSimpleName.stripSuffix("$"))
4643

4744
def initTable(spark: SparkSession): Seq[TableGenerator] = {
4845
import spark.implicits._
@@ -597,52 +594,50 @@ object DataGenerator {
597594
promotion)
598595
}
599596

600-
def run(config: RunConfig): Unit = {
601-
val conf = new SparkConf()
602-
val db = conf.get(DATABASE_KEY, config.db)
603-
val scaleFactor = conf.get(DataGenerator.SCALE_FACTOR_KEY, config.scaleFactor.toString).toInt
604-
597+
def run(config: Config): Unit = {
605598
val spark = SparkSession.builder()
606-
.appName(s"Kyuubi TPCDS Generation - ${scaleFactor}GB")
607-
.config(conf)
599+
.appName(s"Kyuubi TPC-DS Generation - ${config.scaleFactor}GB")
608600
.enableHiveSupport()
609601
.getOrCreate()
610602

611-
logger.info(s"Generating TPCDS tables under database $db")
612-
spark.sql(s"CREATE DATABASE IF NOT EXISTS $db")
613-
spark.sql(s"USE $db")
614-
spark.sql(s"DESC DATABASE $db").show()
615-
616-
val parallel = spark.sparkContext.defaultParallelism
603+
logger.info(s"Generating TPC-DS tables under database ${config.db}")
604+
spark.sql(s"CREATE DATABASE IF NOT EXISTS ${config.db}")
605+
spark.sql(s"USE ${config.db}")
606+
spark.sql(s"DESC DATABASE ${config.db}").show()
617607

618608
val tpcdsTables = initTable(spark)
619609
tpcdsTables.par.foreach { table =>
620-
table.setScaleFactor(scaleFactor)
621-
table.setParallelism(parallel)
610+
table.setScaleFactor(config.scaleFactor)
611+
table.setFormat(config.format)
612+
config.parallel.foreach(table.setParallelism)
622613
spark.sparkContext.setJobDescription(table.toString)
623-
logger.info(s"$table")
614+
logger.info(s"Generating $table")
624615
table.create()
625616
}
626617
}
627618

628619
def main(args: Array[String]): Unit = {
629-
val parser = new scopt.OptionParser[RunConfig]("tpcds-data-generator") {
630-
head("tpcds-data-generator", "")
620+
val parser = new scopt.OptionParser[Config]("tpcds-data-generator") {
621+
head("Kyuubi TPC-DS Data Generator")
631622
opt[String]('d', "db")
632623
.action { (x, c) => c.copy(db = x) }
633-
.text("the databases to write data")
624+
.text("the database to write data")
634625
opt[Int]('s', "scaleFactor")
635626
.action { (x, c) => c.copy(scaleFactor = x) }
636-
.text("the scale factor of tpcds")
637-
help("help")
627+
.text("the scale factor of TPC-DS")
628+
opt[String]('f', "format")
629+
.action { (x, c) => c.copy(format = x) }
630+
.text("the format of table to store data")
631+
opt[Int]('p', "parallel")
632+
.action { (x, c) => c.copy(parallel = Some(x)) }
633+
.text("the parallelism of Spark job")
634+
help('h', "help")
638635
.text("prints this usage text")
639636
}
640637

641-
parser.parse(args, RunConfig()) match {
642-
case Some(config) =>
643-
run(config)
644-
case None =>
645-
System.exit(1)
638+
parser.parse(args, Config()) match {
639+
case Some(config) => run(config)
640+
case None => sys.exit(1)
646641
}
647642
}
648643
}

dev/kyuubi-tpcds/src/main/scala/org/apache/kyuubi/tpcds/TableGenerator.scala

Lines changed: 10 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -40,15 +40,18 @@ case class TableGenerator(
4040
private val rawSchema: StructType = StructType(fields.map(f => StructField(f.name, StringType)))
4141

4242
private var scaleFactor: Int = 1
43+
def setScaleFactor(scale: Int): Unit = this.scaleFactor = scale
4344

44-
private var parallelism: Int = scaleFactor * 2
45+
private var _parallelism: Option[Int] = None
46+
private def parallelism: Int = _parallelism.getOrElse(scaleFactor * 2)
47+
def setParallelism(parallel: Int): Unit = this._parallelism = Some(parallel max 2)
4548

4649
private val ss: SparkSession = SparkSession.active
47-
private val format: String = ss.conf.get("spark.sql.sources.default", "parquet")
50+
private var _format: Option[String] = None
51+
private def format: String = _format.getOrElse(ss.conf.get("spark.sql.sources.default"))
52+
def setFormat(format: String): Unit = this._format = Some(format)
4853

49-
private def radix: Int = {
50-
math.min(math.max(5, scaleFactor / 100), parallelism)
51-
}
54+
private def radix: Int = (scaleFactor / 100) max 5 min parallelism
5255

5356
private def toDF: DataFrame = {
5457
val rawRDD = ss.sparkContext.parallelize(1 to parallelism, parallelism).flatMap { i =>
@@ -101,8 +104,8 @@ case class TableGenerator(
101104
val data = Paths.get(tempDir.toString, s"${name}_${i}_$parallelism.dat")
102105
val iterator =
103106
if (Files.exists(data)) {
104-
// ... realized that when opening the dat files I should use the “Cp1252” encoding.
105-
// https://github.com/databricks/spark-sql-perf/pull/104
107+
// The data generated by `dsdgen` encoding in "Cp1252".
108+
// See detail at https://github.com/databricks/spark-sql-perf/pull/104
106109
// noinspection SourceNotClosed
107110
Source.fromFile(data.toFile, "cp1252", 8192).getLines
108111
} else {
@@ -123,14 +126,6 @@ case class TableGenerator(
123126
ss.createDataFrame(rowRDD, rawSchema).select(columns: _*)
124127
}
125128

126-
def setScaleFactor(scale: Int): Unit = {
127-
this.scaleFactor = scale
128-
}
129-
130-
def setParallelism(parallel: Int): Unit = {
131-
this.parallelism = math.max(2, parallel)
132-
}
133-
134129
def create(): Unit = {
135130
val data =
136131
if (partitionCols.isEmpty) {

dev/kyuubi-tpcds/src/main/scala/org/apache/kyuubi/tpcds/benchmark/Benchmark.scala

Lines changed: 17 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ abstract class Benchmark(
3636

3737
import Benchmark._
3838

39-
val resultsLocation =
39+
val resultsLocation: String =
4040
sparkSession.conf.get(
4141
"spark.sql.perf.results",
4242
"/spark/sql/performance")
@@ -45,15 +45,16 @@ abstract class Benchmark(
4545

4646
implicit protected def toOption[A](a: A): Option[A] = Option(a)
4747

48-
val buildInfo = Try(getClass.getClassLoader.loadClass("org.apache.spark.BuildInfo")).map { cls =>
49-
cls.getMethods
50-
.filter(_.getReturnType == classOf[String])
51-
.filterNot(_.getName == "toString")
52-
.map(m => m.getName -> m.invoke(cls).asInstanceOf[String])
53-
.toMap
54-
}.getOrElse(Map.empty)
48+
val buildInfo: Map[String, String] =
49+
Try(getClass.getClassLoader.loadClass("org.apache.spark.BuildInfo")).map { cls =>
50+
cls.getMethods
51+
.filter(_.getReturnType == classOf[String])
52+
.filterNot(_.getName == "toString")
53+
.map(m => m.getName -> m.invoke(cls).asInstanceOf[String])
54+
.toMap
55+
}.getOrElse(Map.empty)
5556

56-
def currentConfiguration = BenchmarkConfiguration(
57+
def currentConfiguration: BenchmarkConfiguration = BenchmarkConfiguration(
5758
sqlConf = sparkSession.conf.getAll,
5859
sparkConf = sparkContext.getConf.getAll.toMap,
5960
defaultParallelism = sparkContext.defaultParallelism,
@@ -82,7 +83,7 @@ abstract class Benchmark(
8283
tags: Map[String, String] = Map.empty,
8384
timeout: Long = 0L,
8485
resultLocation: String = resultsLocation,
85-
forkThread: Boolean = true) = {
86+
forkThread: Boolean = true): ExperimentStatus = {
8687

8788
new ExperimentStatus(
8889
executionsToRun,
@@ -150,7 +151,7 @@ object Benchmark {
150151
val currentRuns = new collection.mutable.ArrayBuffer[ExperimentRun]()
151152
val currentMessages = new collection.mutable.ArrayBuffer[String]()
152153

153-
def logMessage(msg: String) = {
154+
def logMessage(msg: String): Unit = {
154155
println(msg)
155156
currentMessages += msg
156157
}
@@ -170,10 +171,11 @@ object Benchmark {
170171
case h :: t => for (xh <- h; xt <- cartesianProduct(t)) yield xh :: xt
171172
}
172173

173-
val timestamp = System.currentTimeMillis()
174+
val timestamp: Long = System.currentTimeMillis()
174175
val resultPath = s"$resultsLocation/timestamp=$timestamp"
175-
val combinations = cartesianProduct(variations.map(l => (0 until l.options.size).toList).toList)
176-
val resultsFuture = Future {
176+
val combinations: Seq[List[Int]] =
177+
cartesianProduct(variations.map(l => l.options.indices.toList).toList)
178+
val resultsFuture: Future[Unit] = Future {
177179
// Run the benchmarks!
178180
val results: Seq[ExperimentRun] = (1 to iterations).flatMap { i =>
179181
combinations.map { setup =>
@@ -260,7 +262,7 @@ object Benchmark {
260262
}
261263

262264
/** Waits for the finish of the experiment. */
263-
def waitForFinish(timeoutInSeconds: Int) = {
265+
def waitForFinish(timeoutInSeconds: Int): Unit = {
264266
Await.result(resultsFuture, timeoutInSeconds.seconds)
265267
}
266268

dev/kyuubi-tpcds/src/main/scala/org/apache/kyuubi/tpcds/benchmark/Query.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ class Query(
5353
}
5454
}
5555

56-
lazy val tablesInvolved = buildDataFrame.queryExecution.logical collect {
56+
lazy val tablesInvolved: Seq[String] = buildDataFrame.queryExecution.logical collect {
5757
case r: UnresolvedRelation => r.tableName
5858
}
5959

dev/kyuubi-tpcds/src/main/scala/org/apache/kyuubi/tpcds/benchmark/RunBenchmark.scala

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -34,9 +34,11 @@ case class RunConfig(
3434
/**
3535
* Usage:
3636
* <p>
37-
* Run following command to benchmark tpcds sf10 with exists database `tpcds_sf10`.
37+
* Run following command to benchmark TPC-DS sf10 with exists database `tpcds_sf10`.
3838
* {{{
39-
* $SPARK_HOME/bin/spark-submit --class org.apache.kyuubi.tpcds.benchmark.RunBenchmark kyuubi-tpcds-*.jar --db tpcds_sf10
39+
* $SPARK_HOME/bin/spark-submit \
40+
* --class org.apache.kyuubi.tpcds.benchmark.RunBenchmark \
41+
* kyuubi-tpcds_*.jar --db tpcds_sf10
4042
* }}}
4143
*/
4244
object RunBenchmark {
@@ -61,10 +63,8 @@ object RunBenchmark {
6163
}
6264

6365
parser.parse(args, RunConfig()) match {
64-
case Some(config) =>
65-
run(config)
66-
case None =>
67-
System.exit(1)
66+
case Some(config) => run(config)
67+
case None => sys.exit(1)
6868
}
6969
}
7070

@@ -95,7 +95,7 @@ object RunBenchmark {
9595
val experiment = benchmark.runExperiment(
9696
executionsToRun = allQueries,
9797
iterations = config.iterations,
98-
tags = Map("host" -> InetAddress.getLocalHost().getHostName()))
98+
tags = Map("host" -> InetAddress.getLocalHost.getHostName))
9999

100100
println("== STARTING EXPERIMENT ==")
101101
experiment.waitForFinish(1000 * 60 * 30)

dev/kyuubi-tpcds/src/main/scala/org/apache/kyuubi/tpcds/benchmark/TPCDS.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,5 +24,5 @@ import org.apache.spark.sql.SparkSession
2424
*/
2525
class TPCDS(@transient sparkSession: SparkSession)
2626
extends Benchmark(sparkSession)
27-
with Tpcds_2_4_Queries
27+
with TPCDS_2_4_Queries
2828
with Serializable {}

0 commit comments

Comments
 (0)