You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "LuciferYang (via GitHub)" <gi...@apache.org> on 2023/07/06 08:06:30 UTC

[GitHub] [spark] LuciferYang commented on a diff in pull request #41856: [SPARK-44301][SQL] Add Benchmark Suite for TPCH

LuciferYang commented on code in PR #41856:
URL: https://github.com/apache/spark/pull/41856#discussion_r1254038964


##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCHData.scala:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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
+
+import scala.util.Try
+
+import org.apache.spark.SparkContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.functions.{col, rpad}
+import org.apache.spark.sql.types.{CharType, StringType, StructField, StructType, VarcharType}
+
+// The classes in this file are basically moved from https://github.com/databricks/spark-sql-perf

Review Comment:
   Based on [TPCH.scala](https://github.com/databricks/spark-sql-perf/blob/master/src/main/scala/com/databricks/spark/sql/perf/tpch/TPCH.scala)?



##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCHData.scala:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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
+
+import scala.util.Try
+
+import org.apache.spark.SparkContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.functions.{col, rpad}
+import org.apache.spark.sql.types.{CharType, StringType, StructField, StructType, VarcharType}
+
+// The classes in this file are basically moved from https://github.com/databricks/spark-sql-perf
+
+class Dbgen(dbgenDir: String, params: Seq[String]) extends Serializable {
+  val dbgen = s"$dbgenDir/dbgen"
+  def generate(
+    sparkContext: SparkContext,
+    tableName: String,
+    partitions: Int,
+    scaleFactor: Int): RDD[String] = {
+    val generatedData = {
+      sparkContext.parallelize(1 to partitions, partitions).flatMap { i =>
+        val localToolsDir = if (new java.io.File(dbgen).exists) {
+          dbgenDir
+        } else if (new java.io.File(s"/$dbgenDir").exists) {
+          s"/$dbgenDir"
+        } else {
+          throw new IllegalStateException
+            (s"Could not find dbgen at $dbgen or /$dbgenDir. Run install")
+        }
+        val parallel = if (partitions > 1) s"-C $partitions -S $i" else ""
+        val shortTableNames = Map(
+          "customer" -> "c",
+          "lineitem" -> "L",
+          "nation" -> "n",
+          "orders" -> "O",
+          "part" -> "P",
+          "region" -> "r",
+          "supplier" -> "s",
+          "partsupp" -> "S"
+        )
+        val paramsString = params.mkString(" ")
+        val commands = Seq(
+          "bash", "-c",
+          s"cd $localToolsDir && ./dbgen -q $paramsString " +
+            s"-T ${shortTableNames(tableName)} -s $scaleFactor $parallel")
+        BlockingLineStream(commands)
+      }
+    }
+
+    generatedData.setName(s"$tableName, sf=$scaleFactor, strings")
+    generatedData
+  }
+}
+
+class TPCHTables(
+    sqlContext: SQLContext,
+    dbgenDir: String,
+    scaleFactor: Int,
+    generatorParams: Seq[String] = Nil)
+    extends TPCHSchema with Logging with Serializable {
+
+  private val dataGenerator = new Dbgen(dbgenDir, generatorParams)
+
+  private def tables: Seq[Table] = tableColumns.map { case (tableName, schemaString) =>
+    val partitionColumns = tablePartitionColumns.getOrElse(tableName, Nil)
+      .map(_.stripPrefix("`").stripSuffix("`"))
+    Table(tableName, partitionColumns, StructType.fromDDL(schemaString))
+  }.toSeq
+
+  private case class Table(name: String, partitionColumns: Seq[String], schema: StructType) {
+    def nonPartitioned: Table = {
+      Table(name, Nil, schema)
+    }
+
+    private def df(numPartition: Int) = {
+      val generatedData = dataGenerator.generate(
+        sqlContext.sparkContext, name, numPartition, scaleFactor)
+      val rows = generatedData.mapPartitions { iter =>
+        iter.map { l =>
+          val values = l.split("\\|", -1).dropRight(1).map { v =>
+            if (v.equals("")) {
+              // If the string value is an empty string, we turn it to a null
+              null
+            } else {
+              v
+            }
+          }
+          Row.fromSeq(values)
+        }
+      }
+
+      val stringData =
+        sqlContext.createDataFrame(
+          rows,
+          StructType(schema.fields.map(f => StructField(f.name, StringType))))
+
+      val convertedData = {
+        val columns = schema.fields.map { f =>
+          val c = f.dataType match {
+            // Needs right-padding for char types
+            case CharType(n) => rpad(Column(f.name), n, " ")
+            // Don't need a cast for varchar types
+            case _: VarcharType => col(f.name)
+            case _ => col(f.name).cast(f.dataType)
+          }
+          c.as(f.name)
+        }
+        stringData.select(columns: _*)
+      }
+
+      convertedData
+    }
+
+    def genData(
+        location: String,
+        format: String,
+        overwrite: Boolean,
+        clusterByPartitionColumns: Boolean,
+        filterOutNullPartitionValues: Boolean,
+        numPartitions: Int): Unit = {
+      val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Ignore
+
+      val data = df(numPartitions)
+      val tempTableName = s"${name}_text"
+      data.createOrReplaceTempView(tempTableName)
+
+      val writer = if (partitionColumns.nonEmpty) {
+        if (clusterByPartitionColumns) {
+          val columnString = data.schema.fields.map { field =>
+            field.name
+          }.mkString(",")
+          val partitionColumnString = partitionColumns.mkString(",")
+          val predicates = if (filterOutNullPartitionValues) {
+            partitionColumns.map(col => s"$col IS NOT NULL").mkString("WHERE ", " AND ", "")
+          } else {
+            ""
+          }
+
+          val query =
+            s"""
+               |SELECT
+               |  $columnString
+               |FROM
+               |  $tempTableName
+               |$predicates
+               |DISTRIBUTE BY
+               |  $partitionColumnString
+            """.stripMargin
+          val grouped = sqlContext.sql(query)
+          logInfo(s"Pre-clustering with partitioning columns with query $query.")
+          grouped.write
+        } else {
+          data.write
+        }
+      } else {
+        // treat non-partitioned tables as "one partition" that we want to coalesce
+        if (clusterByPartitionColumns) {
+          // in case data has more than maxRecordsPerFile, split into multiple writers to improve
+          // datagen speed files will be truncated to maxRecordsPerFile value, so the final
+          // result will be the same.
+          val numRows = data.count
+          val maxRecordPerFile = Try {
+            sqlContext.getConf("spark.sql.files.maxRecordsPerFile").toInt
+          }.getOrElse(0)
+
+          if (maxRecordPerFile > 0 && numRows > maxRecordPerFile) {
+            val numFiles = (numRows.toDouble/maxRecordPerFile).ceil.toInt
+            logInfo(s"Coalescing into $numFiles files")
+            data.coalesce(numFiles).write
+          } else {
+            data.coalesce(1).write
+          }
+        } else {
+          data.write
+        }
+      }
+      writer.format(format).mode(mode)
+      if (partitionColumns.nonEmpty) {
+        writer.partitionBy(partitionColumns: _*)
+      }
+      logInfo(s"Generating table $name in database to $location with save mode $mode.")
+      writer.save(location)
+      sqlContext.dropTempTable(tempTableName)
+    }
+  }
+
+  def genData(
+      location: String,
+      format: String,
+      overwrite: Boolean,
+      partitionTables: Boolean,
+      clusterByPartitionColumns: Boolean,
+      filterOutNullPartitionValues: Boolean,
+      tableFilter: String = "",
+      numPartitions: Int = 100): Unit = {
+    var tablesToBeGenerated = if (partitionTables) {
+      tables
+    } else {
+      tables.map(_.nonPartitioned)
+    }
+
+    if (!tableFilter.isEmpty) {
+      tablesToBeGenerated = tablesToBeGenerated.filter(_.name == tableFilter)
+      if (tablesToBeGenerated.isEmpty) {
+        throw new RuntimeException("Bad table name filter: " + tableFilter)
+      }
+    }
+
+    tablesToBeGenerated.foreach { table =>
+      val tableLocation = s"$location/${table.name}"
+      table.genData(tableLocation, format, overwrite, clusterByPartitionColumns,
+        filterOutNullPartitionValues, numPartitions)
+    }
+  }
+}
+
+class GenTPCHDataConfig(args: Array[String]) {
+  var master: String = "local[*]"
+  var dbgenDir: String = null
+  var location: String = null
+  var scaleFactor: Int = 1
+  var format: String = "parquet"
+  var overwrite: Boolean = false
+  var partitionTables: Boolean = false
+  var clusterByPartitionColumns: Boolean = false
+  var filterOutNullPartitionValues: Boolean = false
+  var tableFilter: String = ""
+  var numPartitions: Int = 100
+
+  parseArgs(args.toList)
+
+  private def parseArgs(inputArgs: List[String]): Unit = {
+    var args = inputArgs
+
+    while (args.nonEmpty) {
+      args match {
+        case "--master" :: value :: tail =>
+          master = value
+          args = tail
+
+        case "--dbgenDir" :: value :: tail =>
+          dbgenDir = value
+          args = tail
+
+        case "--location" :: value :: tail =>
+          location = value
+          args = tail
+
+        case "--scaleFactor" :: value :: tail =>
+          scaleFactor = toPositiveIntValue("Scale factor", value)
+          args = tail
+
+        case "--format" :: value :: tail =>
+          format = value
+          args = tail
+
+        case "--overwrite" :: tail =>
+          overwrite = true
+          args = tail
+
+        case "--partitionTables" :: tail =>
+          partitionTables = true
+          args = tail
+
+        case "--clusterByPartitionColumns" :: tail =>
+          clusterByPartitionColumns = true
+          args = tail
+
+        case "--filterOutNullPartitionValues" :: tail =>
+          filterOutNullPartitionValues = true
+          args = tail
+
+        case "--tableFilter" :: value :: tail =>
+          tableFilter = value
+          args = tail
+
+        case "--numPartitions" :: value :: tail =>
+          numPartitions = toPositiveIntValue("Number of partitions", value)
+          args = tail
+
+        case "--help" :: tail =>
+          printUsageAndExit(0)
+
+        case _ =>
+          // scalastyle:off println
+          System.err.println("Unknown/unsupported param " + args)
+          // scalastyle:on println
+          printUsageAndExit(1)
+      }
+    }
+
+    checkRequiredArguments()
+  }
+
+  private def printUsageAndExit(exitCode: Int): Unit = {
+    // scalastyle:off
+    System.err.println("""
+                         |build/sbt "test:runMain <this class> [Options]"

Review Comment:
   line 315 ~ 328
   indentation: 2 spaces
   
   And this file seems have significant duplication with https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala. 
   
   Is it possible to further refactor to reduce duplicate code?
   
   



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCQueryBenchmarkArguments.scala:
##########
@@ -20,14 +20,11 @@ package org.apache.spark.sql.execution.benchmark
 import java.util.Locale
 
 
-class TPCDSQueryBenchmarkArguments(val args: Array[String]) {
-  var dataLocation: String = sys.env.getOrElse("SPARK_TPCDS_DATA", null)
+class TPCQueryBenchmarkArguments(val args: Array[String], val dataLocationEnv: String) {
+  var dataLocation: String = sys.env.getOrElse(dataLocationEnv, null)
   var queryFilter: Set[String] = Set.empty
   var cboEnabled: Boolean = false
 
-  parseArgs(args.toList)

Review Comment:
   why remove the invoke of `parseArgs` and `validateArguments`, these 2 functions will be unused after this pr



##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCHData.scala:
##########
@@ -0,0 +1,401 @@
+/*
+ * 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
+
+import scala.util.Try
+
+import org.apache.spark.SparkContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.functions.{col, rpad}
+import org.apache.spark.sql.types.{CharType, StringType, StructField, StructType, VarcharType}
+
+// The classes in this file are basically moved from https://github.com/databricks/spark-sql-perf
+
+class Dbgen(dbgenDir: String, params: Seq[String]) extends Serializable {
+  val dbgen = s"$dbgenDir/dbgen"
+  def generate(
+    sparkContext: SparkContext,

Review Comment:
   line 33 ~ 36
   indentation: 4 spaces



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCBenchmarkUtils.scala:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.benchmark
+
+import scala.util.Try
+
+import org.apache.spark.SparkConf
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
+import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias
+import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND
+import org.apache.spark.sql.catalyst.util.resourceToString
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+
+
+trait TPCBenchmarkUtils extends SqlBasedBenchmark with Logging {

Review Comment:
   suggested a different name, maybe TPCBasedBenchmark?  may be other more suitable ones



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org