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


##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCData.scala:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 java.util.concurrent.LinkedBlockingQueue
+
+import scala.sys.process._
+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
+
+/**
+ * Using ProcessBuilder.lineStream produces a stream, that uses
+ * a LinkedBlockingQueue with a default capacity of Integer.MAX_VALUE.
+ *
+ * This causes OOM if the consumer cannot keep up with the producer.
+ *
+ * See scala.sys.process.ProcessBuilderImpl.lineStream
+ */
+object BlockingLineStream {
+
+  // See scala.sys.process.Streamed
+  private final class BlockingStreamed[T](
+      val process: T => Unit,
+      val done: Int => Unit,
+      val stream: () => Stream[T])
+
+  // See scala.sys.process.Streamed
+  private object BlockingStreamed {
+    // scala.process.sys.Streamed uses default of Integer.MAX_VALUE,
+    // which causes OOMs if the consumer cannot keep up with producer.
+    val maxQueueSize = 65536
+
+    def apply[T](nonzeroException: Boolean): BlockingStreamed[T] = {
+      val q = new LinkedBlockingQueue[Either[Int, T]](maxQueueSize)
+
+      def next(): Stream[T] = q.take match {
+        case Left(0) => Stream.empty
+        case Left(code) =>
+          if (nonzeroException) scala.sys.error("Nonzero exit code: " + code) 
else Stream.empty
+        case Right(s) => Stream.cons(s, next())
+      }
+
+      new BlockingStreamed((s: T) => q put Right(s), code => q put Left(code), 
() => next())
+    }
+  }
+
+  // See scala.sys.process.ProcessImpl.Spawn
+  private object Spawn {
+    def apply(f: => Unit): Thread = apply(f, daemon = false)
+    def apply(f: => Unit, daemon: Boolean): Thread = {
+      val thread = new Thread() { override def run() = { f } }
+      thread.setDaemon(daemon)
+      thread.start()
+      thread
+    }
+  }
+
+  def apply(command: Seq[String]): Stream[String] = {
+    val streamed = BlockingStreamed[String](true)
+    val process = command.run(BasicIO(false, streamed.process, None))
+    Spawn(streamed.done(process.exitValue()))
+    streamed.stream()
+  }
+}
+
+trait DataGenerator extends Serializable {
+  protected val toolsDir: String
+
+  protected def generateData(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int,
+      command: String): RDD[String] = {
+    val generatedData = {
+      sparkContext.parallelize(1 to partitions, partitions).flatMap { i =>
+        val localToolsDir = if (new java.io.File(toolsDir).exists) {
+          toolsDir
+        } else if (new java.io.File(s"/$toolsDir").exists) {
+          s"/$toolsDir"
+        } else {
+          throw new IllegalStateException(
+            s"Could not find tools at $toolsDir or /$toolsDir. Run install")
+        }
+
+        val parallel = if (partitions > 1) s"-parallel $partitions -child $i" 
else ""
+        val commands = Seq("bash", "-c", s"cd $localToolsDir && $command 
$parallel")
+        BlockingLineStream(commands)
+      }
+    }
+
+    generatedData.setName(s"$tableName, sf=$scaleFactor, strings")
+    generatedData
+  }
+
+  def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String]
+}
+
+class Dsdgen(dsdgenDir: String) extends DataGenerator with Serializable {
+  override protected val toolsDir: String = s"$dsdgenDir"
+
+  override def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String] = {
+    val command = s"./dsdgen -table $tableName -filter Y -scale $scaleFactor 
-RNGSEED 100"
+    generateData(sparkContext, tableName, partitions, scaleFactor, command)
+  }
+}
+
+class Dbgen(dbgenDir: String, params: Seq[String])
+  extends DataGenerator with Serializable {
+  override protected val toolsDir: String = s"$dbgenDir"
+  override def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String] = {
+    val shortTableNames = Map(
+      "customer" -> "c",
+      "lineitem" -> "L",
+      "nation" -> "n",
+      "orders" -> "O",
+      "part" -> "P",
+      "region" -> "r",
+      "supplier" -> "s",
+      "partsupp" -> "S"
+    )
+    val paramsString = params.mkString(" ")
+    val command = s"./dbgen -q $paramsString -T ${shortTableNames(tableName)} 
-s $scaleFactor"
+    generateData(sparkContext, tableName, partitions, scaleFactor, command)
+  }
+}
+
+trait TableGenerator extends Serializable with Logging {
+  protected val dataGenerator: DataGenerator
+  protected val sparkSQLContext: SQLContext

Review Comment:
   ```suggestion
     protected val sqlContext: SQLContext
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCData.scala:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 java.util.concurrent.LinkedBlockingQueue
+
+import scala.sys.process._
+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
+
+/**
+ * Using ProcessBuilder.lineStream produces a stream, that uses
+ * a LinkedBlockingQueue with a default capacity of Integer.MAX_VALUE.
+ *
+ * This causes OOM if the consumer cannot keep up with the producer.
+ *
+ * See scala.sys.process.ProcessBuilderImpl.lineStream
+ */
+object BlockingLineStream {
+
+  // See scala.sys.process.Streamed
+  private final class BlockingStreamed[T](
+      val process: T => Unit,
+      val done: Int => Unit,
+      val stream: () => Stream[T])
+
+  // See scala.sys.process.Streamed
+  private object BlockingStreamed {
+    // scala.process.sys.Streamed uses default of Integer.MAX_VALUE,
+    // which causes OOMs if the consumer cannot keep up with producer.
+    val maxQueueSize = 65536
+
+    def apply[T](nonzeroException: Boolean): BlockingStreamed[T] = {
+      val q = new LinkedBlockingQueue[Either[Int, T]](maxQueueSize)
+
+      def next(): Stream[T] = q.take match {
+        case Left(0) => Stream.empty
+        case Left(code) =>
+          if (nonzeroException) scala.sys.error("Nonzero exit code: " + code) 
else Stream.empty
+        case Right(s) => Stream.cons(s, next())
+      }
+
+      new BlockingStreamed((s: T) => q put Right(s), code => q put Left(code), 
() => next())
+    }
+  }
+
+  // See scala.sys.process.ProcessImpl.Spawn
+  private object Spawn {
+    def apply(f: => Unit): Thread = apply(f, daemon = false)
+    def apply(f: => Unit, daemon: Boolean): Thread = {
+      val thread = new Thread() { override def run() = { f } }
+      thread.setDaemon(daemon)
+      thread.start()
+      thread
+    }
+  }
+
+  def apply(command: Seq[String]): Stream[String] = {
+    val streamed = BlockingStreamed[String](true)
+    val process = command.run(BasicIO(false, streamed.process, None))
+    Spawn(streamed.done(process.exitValue()))
+    streamed.stream()
+  }
+}
+
+trait DataGenerator extends Serializable {
+  protected val toolsDir: String
+
+  protected def generateData(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int,
+      command: String): RDD[String] = {
+    val generatedData = {
+      sparkContext.parallelize(1 to partitions, partitions).flatMap { i =>
+        val localToolsDir = if (new java.io.File(toolsDir).exists) {
+          toolsDir
+        } else if (new java.io.File(s"/$toolsDir").exists) {
+          s"/$toolsDir"
+        } else {
+          throw new IllegalStateException(
+            s"Could not find tools at $toolsDir or /$toolsDir. Run install")
+        }
+
+        val parallel = if (partitions > 1) s"-parallel $partitions -child $i" 
else ""
+        val commands = Seq("bash", "-c", s"cd $localToolsDir && $command 
$parallel")
+        BlockingLineStream(commands)
+      }
+    }
+
+    generatedData.setName(s"$tableName, sf=$scaleFactor, strings")
+    generatedData
+  }
+
+  def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String]
+}
+
+class Dsdgen(dsdgenDir: String) extends DataGenerator with Serializable {
+  override protected val toolsDir: String = s"$dsdgenDir"
+
+  override def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String] = {
+    val command = s"./dsdgen -table $tableName -filter Y -scale $scaleFactor 
-RNGSEED 100"
+    generateData(sparkContext, tableName, partitions, scaleFactor, command)
+  }
+}
+
+class Dbgen(dbgenDir: String, params: Seq[String])
+  extends DataGenerator with Serializable {
+  override protected val toolsDir: String = s"$dbgenDir"
+  override def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String] = {
+    val shortTableNames = Map(
+      "customer" -> "c",
+      "lineitem" -> "L",
+      "nation" -> "n",
+      "orders" -> "O",
+      "part" -> "P",
+      "region" -> "r",
+      "supplier" -> "s",
+      "partsupp" -> "S"
+    )
+    val paramsString = params.mkString(" ")
+    val command = s"./dbgen -q $paramsString -T ${shortTableNames(tableName)} 
-s $scaleFactor"
+    generateData(sparkContext, tableName, partitions, scaleFactor, command)
+  }
+}
+
+trait TableGenerator extends Serializable with Logging {
+  protected val dataGenerator: DataGenerator
+  protected val sparkSQLContext: SQLContext
+  protected val tpcScaleFactor: Int

Review Comment:
   ```suggestion
     protected val scaleFactor: Int
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala:
##########
@@ -17,391 +17,20 @@
 
 package org.apache.spark.sql
 
-import java.util.concurrent.LinkedBlockingQueue
-
-import scala.collection.immutable.Stream
-import scala.sys.process._
-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
-
-/**
- * Using ProcessBuilder.lineStream produces a stream, that uses
- * a LinkedBlockingQueue with a default capacity of Integer.MAX_VALUE.
- *
- * This causes OOM if the consumer cannot keep up with the producer.
- *
- * See scala.sys.process.ProcessBuilderImpl.lineStream
- */
-object BlockingLineStream {
-
-  // See scala.sys.process.Streamed
-  private final class BlockingStreamed[T](
-    val process: T => Unit,
-    val done: Int => Unit,
-    val stream: () => Stream[T])
-
-  // See scala.sys.process.Streamed
-  private object BlockingStreamed {
-    // scala.process.sys.Streamed uses default of Integer.MAX_VALUE,
-    // which causes OOMs if the consumer cannot keep up with producer.
-    val maxQueueSize = 65536
-
-    def apply[T](nonzeroException: Boolean): BlockingStreamed[T] = {
-      val q = new LinkedBlockingQueue[Either[Int, T]](maxQueueSize)
-
-      def next(): Stream[T] = q.take match {
-        case Left(0) => Stream.empty
-        case Left(code) =>
-          if (nonzeroException) scala.sys.error("Nonzero exit code: " + code) 
else Stream.empty
-        case Right(s) => Stream.cons(s, next())
-      }
-
-      new BlockingStreamed((s: T) => q put Right(s), code => q put Left(code), 
() => next())
-    }
-  }
-
-  // See scala.sys.process.ProcessImpl.Spawn
-  private object Spawn {
-    def apply(f: => Unit): Thread = apply(f, daemon = false)
-    def apply(f: => Unit, daemon: Boolean): Thread = {
-      val thread = new Thread() { override def run() = { f } }
-      thread.setDaemon(daemon)
-      thread.start()
-      thread
-    }
-  }
-
-  def apply(command: Seq[String]): Stream[String] = {
-    val streamed = BlockingStreamed[String](true)
-    val process = command.run(BasicIO(false, streamed.process, None))
-    Spawn(streamed.done(process.exitValue()))
-    streamed.stream()
-  }
-}
-
-class Dsdgen(dsdgenDir: String) extends Serializable {
-  private val dsdgen = s"$dsdgenDir/dsdgen"
-
-  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(dsdgen).exists) {
-          dsdgenDir
-        } else if (new java.io.File(s"/$dsdgen").exists) {
-          s"/$dsdgenDir"
-        } else {
-          throw new IllegalStateException(
-            s"Could not find dsdgen at $dsdgen or /$dsdgen. Run install")
-        }
-
-        // NOTE: RNGSEED is the RNG seed used by the data generator. Right 
now, it is fixed to 100.
-        val parallel = if (partitions > 1) s"-parallel $partitions -child $i" 
else ""
-        val commands = Seq(
-          "bash", "-c",
-          s"cd $localToolsDir && ./dsdgen -table $tableName -filter Y -scale 
$scaleFactor " +
-          s"-RNGSEED 100 $parallel")
-        BlockingLineStream(commands)
-      }
-    }
-
-    generatedData.setName(s"$tableName, sf=$scaleFactor, strings")
-    generatedData
-  }
-}
+import org.apache.spark.sql.types.StructType
 
 class TPCDSTables(sqlContext: SQLContext, dsdgenDir: String, scaleFactor: Int)
-  extends TPCDSSchema with Logging with Serializable {
+  extends TableGenerator with TPCDSSchema with Logging with Serializable {
 
-  private val dataGenerator = new Dsdgen(dsdgenDir)
-
-  private def tables: Seq[Table] = tableColumns.map { case (tableName, 
schemaString) =>
+  override protected val dataGenerator: DataGenerator = new Dsdgen(dsdgenDir)
+  override protected val sparkSQLContext: SQLContext = sqlContext
+  override protected val tpcScaleFactor: Int = scaleFactor

Review Comment:
   ```suggestion
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala:
##########
@@ -17,391 +17,20 @@
 
 package org.apache.spark.sql
 
-import java.util.concurrent.LinkedBlockingQueue
-
-import scala.collection.immutable.Stream
-import scala.sys.process._
-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
-
-/**
- * Using ProcessBuilder.lineStream produces a stream, that uses
- * a LinkedBlockingQueue with a default capacity of Integer.MAX_VALUE.
- *
- * This causes OOM if the consumer cannot keep up with the producer.
- *
- * See scala.sys.process.ProcessBuilderImpl.lineStream
- */
-object BlockingLineStream {
-
-  // See scala.sys.process.Streamed
-  private final class BlockingStreamed[T](
-    val process: T => Unit,
-    val done: Int => Unit,
-    val stream: () => Stream[T])
-
-  // See scala.sys.process.Streamed
-  private object BlockingStreamed {
-    // scala.process.sys.Streamed uses default of Integer.MAX_VALUE,
-    // which causes OOMs if the consumer cannot keep up with producer.
-    val maxQueueSize = 65536
-
-    def apply[T](nonzeroException: Boolean): BlockingStreamed[T] = {
-      val q = new LinkedBlockingQueue[Either[Int, T]](maxQueueSize)
-
-      def next(): Stream[T] = q.take match {
-        case Left(0) => Stream.empty
-        case Left(code) =>
-          if (nonzeroException) scala.sys.error("Nonzero exit code: " + code) 
else Stream.empty
-        case Right(s) => Stream.cons(s, next())
-      }
-
-      new BlockingStreamed((s: T) => q put Right(s), code => q put Left(code), 
() => next())
-    }
-  }
-
-  // See scala.sys.process.ProcessImpl.Spawn
-  private object Spawn {
-    def apply(f: => Unit): Thread = apply(f, daemon = false)
-    def apply(f: => Unit, daemon: Boolean): Thread = {
-      val thread = new Thread() { override def run() = { f } }
-      thread.setDaemon(daemon)
-      thread.start()
-      thread
-    }
-  }
-
-  def apply(command: Seq[String]): Stream[String] = {
-    val streamed = BlockingStreamed[String](true)
-    val process = command.run(BasicIO(false, streamed.process, None))
-    Spawn(streamed.done(process.exitValue()))
-    streamed.stream()
-  }
-}
-
-class Dsdgen(dsdgenDir: String) extends Serializable {
-  private val dsdgen = s"$dsdgenDir/dsdgen"
-
-  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(dsdgen).exists) {
-          dsdgenDir
-        } else if (new java.io.File(s"/$dsdgen").exists) {
-          s"/$dsdgenDir"
-        } else {
-          throw new IllegalStateException(
-            s"Could not find dsdgen at $dsdgen or /$dsdgen. Run install")
-        }
-
-        // NOTE: RNGSEED is the RNG seed used by the data generator. Right 
now, it is fixed to 100.
-        val parallel = if (partitions > 1) s"-parallel $partitions -child $i" 
else ""
-        val commands = Seq(
-          "bash", "-c",
-          s"cd $localToolsDir && ./dsdgen -table $tableName -filter Y -scale 
$scaleFactor " +
-          s"-RNGSEED 100 $parallel")
-        BlockingLineStream(commands)
-      }
-    }
-
-    generatedData.setName(s"$tableName, sf=$scaleFactor, strings")
-    generatedData
-  }
-}
+import org.apache.spark.sql.types.StructType
 
 class TPCDSTables(sqlContext: SQLContext, dsdgenDir: String, scaleFactor: Int)
-  extends TPCDSSchema with Logging with Serializable {
+  extends TableGenerator with TPCDSSchema with Logging with Serializable {
 
-  private val dataGenerator = new Dsdgen(dsdgenDir)
-
-  private def tables: Seq[Table] = tableColumns.map { case (tableName, 
schemaString) =>
+  override protected val dataGenerator: DataGenerator = new Dsdgen(dsdgenDir)
+  override protected val sparkSQLContext: SQLContext = sqlContext
+  override protected val tpcScaleFactor: Int = scaleFactor

Review Comment:
   Please declare `sqlContext: SQLContext` and `scaleFactor: Int` in 
`TableGenerator`.



##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCData.scala:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 java.util.concurrent.LinkedBlockingQueue
+
+import scala.sys.process._
+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
+
+/**
+ * Using ProcessBuilder.lineStream produces a stream, that uses
+ * a LinkedBlockingQueue with a default capacity of Integer.MAX_VALUE.
+ *
+ * This causes OOM if the consumer cannot keep up with the producer.
+ *
+ * See scala.sys.process.ProcessBuilderImpl.lineStream
+ */
+object BlockingLineStream {
+
+  // See scala.sys.process.Streamed
+  private final class BlockingStreamed[T](
+      val process: T => Unit,
+      val done: Int => Unit,
+      val stream: () => Stream[T])
+
+  // See scala.sys.process.Streamed
+  private object BlockingStreamed {
+    // scala.process.sys.Streamed uses default of Integer.MAX_VALUE,
+    // which causes OOMs if the consumer cannot keep up with producer.
+    val maxQueueSize = 65536
+
+    def apply[T](nonzeroException: Boolean): BlockingStreamed[T] = {
+      val q = new LinkedBlockingQueue[Either[Int, T]](maxQueueSize)
+
+      def next(): Stream[T] = q.take match {
+        case Left(0) => Stream.empty
+        case Left(code) =>
+          if (nonzeroException) scala.sys.error("Nonzero exit code: " + code) 
else Stream.empty
+        case Right(s) => Stream.cons(s, next())
+      }
+
+      new BlockingStreamed((s: T) => q put Right(s), code => q put Left(code), 
() => next())
+    }
+  }
+
+  // See scala.sys.process.ProcessImpl.Spawn
+  private object Spawn {
+    def apply(f: => Unit): Thread = apply(f, daemon = false)
+    def apply(f: => Unit, daemon: Boolean): Thread = {
+      val thread = new Thread() { override def run() = { f } }
+      thread.setDaemon(daemon)
+      thread.start()
+      thread
+    }
+  }
+
+  def apply(command: Seq[String]): Stream[String] = {
+    val streamed = BlockingStreamed[String](true)
+    val process = command.run(BasicIO(false, streamed.process, None))
+    Spawn(streamed.done(process.exitValue()))
+    streamed.stream()
+  }
+}
+
+trait DataGenerator extends Serializable {
+  protected val toolsDir: String
+
+  protected def generateData(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int,
+      command: String): RDD[String] = {
+    val generatedData = {
+      sparkContext.parallelize(1 to partitions, partitions).flatMap { i =>
+        val localToolsDir = if (new java.io.File(toolsDir).exists) {
+          toolsDir
+        } else if (new java.io.File(s"/$toolsDir").exists) {
+          s"/$toolsDir"
+        } else {
+          throw new IllegalStateException(
+            s"Could not find tools at $toolsDir or /$toolsDir. Run install")
+        }
+
+        val parallel = if (partitions > 1) s"-parallel $partitions -child $i" 
else ""
+        val commands = Seq("bash", "-c", s"cd $localToolsDir && $command 
$parallel")
+        BlockingLineStream(commands)
+      }
+    }
+
+    generatedData.setName(s"$tableName, sf=$scaleFactor, strings")
+    generatedData
+  }
+
+  def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String]
+}
+
+class Dsdgen(dsdgenDir: String) extends DataGenerator with Serializable {
+  override protected val toolsDir: String = s"$dsdgenDir"

Review Comment:
   Shall we aovid the duplicated variable `dsdgenDir` and `toolsDir` ?



##########
sql/core/src/test/scala/org/apache/spark/sql/GenTPCData.scala:
##########
@@ -0,0 +1,445 @@
+/*
+ * 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 java.util.concurrent.LinkedBlockingQueue
+
+import scala.sys.process._
+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
+
+/**
+ * Using ProcessBuilder.lineStream produces a stream, that uses
+ * a LinkedBlockingQueue with a default capacity of Integer.MAX_VALUE.
+ *
+ * This causes OOM if the consumer cannot keep up with the producer.
+ *
+ * See scala.sys.process.ProcessBuilderImpl.lineStream
+ */
+object BlockingLineStream {
+
+  // See scala.sys.process.Streamed
+  private final class BlockingStreamed[T](
+      val process: T => Unit,
+      val done: Int => Unit,
+      val stream: () => Stream[T])
+
+  // See scala.sys.process.Streamed
+  private object BlockingStreamed {
+    // scala.process.sys.Streamed uses default of Integer.MAX_VALUE,
+    // which causes OOMs if the consumer cannot keep up with producer.
+    val maxQueueSize = 65536
+
+    def apply[T](nonzeroException: Boolean): BlockingStreamed[T] = {
+      val q = new LinkedBlockingQueue[Either[Int, T]](maxQueueSize)
+
+      def next(): Stream[T] = q.take match {
+        case Left(0) => Stream.empty
+        case Left(code) =>
+          if (nonzeroException) scala.sys.error("Nonzero exit code: " + code) 
else Stream.empty
+        case Right(s) => Stream.cons(s, next())
+      }
+
+      new BlockingStreamed((s: T) => q put Right(s), code => q put Left(code), 
() => next())
+    }
+  }
+
+  // See scala.sys.process.ProcessImpl.Spawn
+  private object Spawn {
+    def apply(f: => Unit): Thread = apply(f, daemon = false)
+    def apply(f: => Unit, daemon: Boolean): Thread = {
+      val thread = new Thread() { override def run() = { f } }
+      thread.setDaemon(daemon)
+      thread.start()
+      thread
+    }
+  }
+
+  def apply(command: Seq[String]): Stream[String] = {
+    val streamed = BlockingStreamed[String](true)
+    val process = command.run(BasicIO(false, streamed.process, None))
+    Spawn(streamed.done(process.exitValue()))
+    streamed.stream()
+  }
+}
+
+trait DataGenerator extends Serializable {
+  protected val toolsDir: String
+
+  protected def generateData(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int,
+      command: String): RDD[String] = {
+    val generatedData = {
+      sparkContext.parallelize(1 to partitions, partitions).flatMap { i =>
+        val localToolsDir = if (new java.io.File(toolsDir).exists) {
+          toolsDir
+        } else if (new java.io.File(s"/$toolsDir").exists) {
+          s"/$toolsDir"
+        } else {
+          throw new IllegalStateException(
+            s"Could not find tools at $toolsDir or /$toolsDir. Run install")
+        }
+
+        val parallel = if (partitions > 1) s"-parallel $partitions -child $i" 
else ""
+        val commands = Seq("bash", "-c", s"cd $localToolsDir && $command 
$parallel")
+        BlockingLineStream(commands)
+      }
+    }
+
+    generatedData.setName(s"$tableName, sf=$scaleFactor, strings")
+    generatedData
+  }
+
+  def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String]
+}
+
+class Dsdgen(dsdgenDir: String) extends DataGenerator with Serializable {
+  override protected val toolsDir: String = s"$dsdgenDir"
+
+  override def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String] = {
+    val command = s"./dsdgen -table $tableName -filter Y -scale $scaleFactor 
-RNGSEED 100"
+    generateData(sparkContext, tableName, partitions, scaleFactor, command)
+  }
+}
+
+class Dbgen(dbgenDir: String, params: Seq[String])
+  extends DataGenerator with Serializable {
+  override protected val toolsDir: String = s"$dbgenDir"
+  override def generate(
+      sparkContext: SparkContext,
+      tableName: String,
+      partitions: Int,
+      scaleFactor: Int): RDD[String] = {
+    val shortTableNames = Map(
+      "customer" -> "c",
+      "lineitem" -> "L",
+      "nation" -> "n",
+      "orders" -> "O",
+      "part" -> "P",
+      "region" -> "r",
+      "supplier" -> "s",
+      "partsupp" -> "S"
+    )
+    val paramsString = params.mkString(" ")
+    val command = s"./dbgen -q $paramsString -T ${shortTableNames(tableName)} 
-s $scaleFactor"
+    generateData(sparkContext, tableName, partitions, scaleFactor, command)
+  }
+}
+
+trait TableGenerator extends Serializable with Logging {
+  protected val dataGenerator: DataGenerator
+  protected val sparkSQLContext: SQLContext

Review Comment:
   BTW. SQLContext is an old API. It seems it's the time use `SparkSession` 
instead. cc @cloud-fan @MaxGekk @wangyum 



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to