Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove merged PR #472: URL: https://github.com/apache/datafusion-comet/pull/472 -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove commented on PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#issuecomment-2145648950 Thanks for the review @viirya -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1624519642 ## fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala: ## @@ -0,0 +1,121 @@ +/* + * 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.comet.fuzz + +import java.io.{BufferedWriter, FileWriter} + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.sql.SparkSession + +object QueryGen { + + def generateRandomQueries( + r: Random, + spark: SparkSession, + numFiles: Int, + numQueries: Int): Unit = { +for (i <- 0 until numFiles) { + spark.read.parquet(s"test$i.parquet").createTempView(s"test$i") +} + +val w = new BufferedWriter(new FileWriter("queries.sql")) + +val uniqueQueries = mutable.HashSet[String]() + +for (_ <- 0 until numQueries) { + val sql = r.nextInt().abs % 3 match { +case 0 => generateJoin(r, spark, numFiles) +case 1 => generateAggregate(r, spark, numFiles) +case 2 => generateScalar(r, spark, numFiles) + } + if (!uniqueQueries.contains(sql)) { +uniqueQueries += sql +w.write(sql + "\n") + } +} +w.close() + } + + private def generateAggregate(r: Random, spark: SparkSession, numFiles: Int): String = { +val tableName = s"test${r.nextInt(numFiles)}" +val table = spark.table(tableName) + +val func = Utils.randomChoice(Meta.aggFunc, r) +val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + +val groupingCols = Range(0, 2).map(_ => Utils.randomChoice(table.columns, r)) Review Comment: If it is valid SQL I think we should allow it -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1623668599 ## fuzz-testing/src/main/scala/org/apache/comet/fuzz/Main.scala: ## @@ -0,0 +1,88 @@ +/* + * 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.comet.fuzz + +import scala.util.Random + +import org.rogach.scallop.{ScallopConf, Subcommand} +import org.rogach.scallop.ScallopOption + +import org.apache.spark.sql.SparkSession + +class Conf(arguments: Seq[String]) extends ScallopConf(arguments) { + object generateData extends Subcommand("data") { +val numFiles: ScallopOption[Int] = opt[Int](required = true) +val numRows: ScallopOption[Int] = opt[Int](required = true) +val numColumns: ScallopOption[Int] = opt[Int](required = true) + } + addSubcommand(generateData) + object generateQueries extends Subcommand("queries") { +val numFiles: ScallopOption[Int] = opt[Int](required = false) +val numQueries: ScallopOption[Int] = opt[Int](required = true) + } + addSubcommand(generateQueries) + object runQueries extends Subcommand("run") { +val filename: ScallopOption[String] = opt[String](required = true) +val numFiles: ScallopOption[Int] = opt[Int](required = false) +val showMatchingResults: ScallopOption[Boolean] = opt[Boolean](required = false) + } + addSubcommand(runQueries) + verify() +} + +object Main { + + lazy val spark: SparkSession = SparkSession +.builder() +.master("local[*]") Review Comment: I have removed this. We specify master during spark-submit so do not need to hardcode it here. -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1623668256 ## fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala: ## @@ -0,0 +1,121 @@ +/* + * 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.comet.fuzz + +import java.io.{BufferedWriter, FileWriter} + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.sql.SparkSession + +object QueryGen { + + def generateRandomQueries( + r: Random, + spark: SparkSession, + numFiles: Int, + numQueries: Int): Unit = { +for (i <- 0 until numFiles) { + spark.read.parquet(s"test$i.parquet").createTempView(s"test$i") +} + +val w = new BufferedWriter(new FileWriter("queries.sql")) + +val uniqueQueries = mutable.HashSet[String]() + +for (_ <- 0 until numQueries) { + val sql = r.nextInt().abs % 3 match { +case 0 => generateJoin(r, spark, numFiles) +case 1 => generateAggregate(r, spark, numFiles) +case 2 => generateScalar(r, spark, numFiles) + } + if (!uniqueQueries.contains(sql)) { +uniqueQueries += sql +w.write(sql + "\n") + } +} +w.close() + } + + private def generateAggregate(r: Random, spark: SparkSession, numFiles: Int): String = { +val tableName = s"test${r.nextInt(numFiles)}" +val table = spark.table(tableName) + +val func = Utils.randomChoice(Meta.aggFunc, r) +val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + +val groupingCols = Range(0, 2).map(_ => Utils.randomChoice(table.columns, r)) + +if (groupingCols.isEmpty) { + s"SELECT ${args.mkString(", ")}, ${func.name}(${args.mkString(", ")}) AS x " + +s"FROM $tableName " + +s"ORDER BY ${args.mkString(", ")};" +} else { + s"SELECT ${groupingCols.mkString(", ")}, ${func.name}(${args.mkString(", ")}) " + +s"FROM $tableName " + +s"GROUP BY ${groupingCols.mkString(",")} " + +s"ORDER BY ${groupingCols.mkString(", ")};" +} + } + + private def generateScalar(r: Random, spark: SparkSession, numFiles: Int): String = { +val tableName = s"test${r.nextInt(numFiles)}" +val table = spark.table(tableName) + +val func = Utils.randomChoice(Meta.scalarFunc, r) +val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + +// Example SELECT c0, log(c0) as x FROM test0 +s"SELECT ${args.mkString(", ")}, ${func.name}(${args.mkString(", ")}) AS x " + + s"FROM $tableName " + + s"ORDER BY ${args.mkString(", ")};" + } + + private def generateJoin(r: Random, spark: SparkSession, numFiles: Int): String = { +val leftTableName = s"test${r.nextInt(numFiles)}" +val rightTableName = s"test${r.nextInt(numFiles)}" +val leftTable = spark.table(leftTableName) +val rightTable = spark.table(rightTableName) + +val leftCol = Utils.randomChoice(leftTable.columns, r) +val rightCol = Utils.randomChoice(rightTable.columns, r) + +val joinTypes = Seq(("INNER", 0.4), ("LEFT", 0.3), ("RIGHT", 0.3)) +val joinType = Utils.randomWeightedChoice(joinTypes) + +val leftColProjection = leftTable.columns.map(c => s"l.$c").mkString(", ") +val rightColProjection = rightTable.columns.map(c => s"r.$c").mkString(", ") +"SELECT " + + s"$leftColProjection, " + + s"$rightColProjection " + + s"FROM $leftTableName l " + + s"$joinType JOIN $rightTableName r " + + s"ON l.$leftCol = r.$rightCol " + Review Comment: Exactly. We just pick random columns and let Spark add implicit casts when needed. -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
viirya commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1622898410 ## fuzz-testing/src/main/scala/org/apache/comet/fuzz/Main.scala: ## @@ -0,0 +1,88 @@ +/* + * 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.comet.fuzz + +import scala.util.Random + +import org.rogach.scallop.{ScallopConf, Subcommand} +import org.rogach.scallop.ScallopOption + +import org.apache.spark.sql.SparkSession + +class Conf(arguments: Seq[String]) extends ScallopConf(arguments) { + object generateData extends Subcommand("data") { +val numFiles: ScallopOption[Int] = opt[Int](required = true) +val numRows: ScallopOption[Int] = opt[Int](required = true) +val numColumns: ScallopOption[Int] = opt[Int](required = true) + } + addSubcommand(generateData) + object generateQueries extends Subcommand("queries") { +val numFiles: ScallopOption[Int] = opt[Int](required = false) +val numQueries: ScallopOption[Int] = opt[Int](required = true) + } + addSubcommand(generateQueries) + object runQueries extends Subcommand("run") { +val filename: ScallopOption[String] = opt[String](required = true) +val numFiles: ScallopOption[Int] = opt[Int](required = false) +val showMatchingResults: ScallopOption[Boolean] = opt[Boolean](required = false) + } + addSubcommand(runQueries) + verify() +} + +object Main { + + lazy val spark: SparkSession = SparkSession +.builder() +.master("local[*]") Review Comment: Do we need to hardcode this? -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
viirya commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1622871985 ## fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala: ## @@ -0,0 +1,121 @@ +/* + * 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.comet.fuzz + +import java.io.{BufferedWriter, FileWriter} + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.sql.SparkSession + +object QueryGen { + + def generateRandomQueries( + r: Random, + spark: SparkSession, + numFiles: Int, + numQueries: Int): Unit = { +for (i <- 0 until numFiles) { + spark.read.parquet(s"test$i.parquet").createTempView(s"test$i") +} + +val w = new BufferedWriter(new FileWriter("queries.sql")) + +val uniqueQueries = mutable.HashSet[String]() + +for (_ <- 0 until numQueries) { + val sql = r.nextInt().abs % 3 match { +case 0 => generateJoin(r, spark, numFiles) +case 1 => generateAggregate(r, spark, numFiles) +case 2 => generateScalar(r, spark, numFiles) + } + if (!uniqueQueries.contains(sql)) { +uniqueQueries += sql +w.write(sql + "\n") + } +} +w.close() + } + + private def generateAggregate(r: Random, spark: SparkSession, numFiles: Int): String = { +val tableName = s"test${r.nextInt(numFiles)}" +val table = spark.table(tableName) + +val func = Utils.randomChoice(Meta.aggFunc, r) +val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + +val groupingCols = Range(0, 2).map(_ => Utils.randomChoice(table.columns, r)) + +if (groupingCols.isEmpty) { + s"SELECT ${args.mkString(", ")}, ${func.name}(${args.mkString(", ")}) AS x " + +s"FROM $tableName " + +s"ORDER BY ${args.mkString(", ")};" +} else { + s"SELECT ${groupingCols.mkString(", ")}, ${func.name}(${args.mkString(", ")}) " + +s"FROM $tableName " + +s"GROUP BY ${groupingCols.mkString(",")} " + +s"ORDER BY ${groupingCols.mkString(", ")};" +} + } + + private def generateScalar(r: Random, spark: SparkSession, numFiles: Int): String = { +val tableName = s"test${r.nextInt(numFiles)}" +val table = spark.table(tableName) + +val func = Utils.randomChoice(Meta.scalarFunc, r) +val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + +// Example SELECT c0, log(c0) as x FROM test0 +s"SELECT ${args.mkString(", ")}, ${func.name}(${args.mkString(", ")}) AS x " + + s"FROM $tableName " + + s"ORDER BY ${args.mkString(", ")};" + } + + private def generateJoin(r: Random, spark: SparkSession, numFiles: Int): String = { +val leftTableName = s"test${r.nextInt(numFiles)}" +val rightTableName = s"test${r.nextInt(numFiles)}" +val leftTable = spark.table(leftTableName) +val rightTable = spark.table(rightTableName) + +val leftCol = Utils.randomChoice(leftTable.columns, r) +val rightCol = Utils.randomChoice(rightTable.columns, r) + +val joinTypes = Seq(("INNER", 0.4), ("LEFT", 0.3), ("RIGHT", 0.3)) +val joinType = Utils.randomWeightedChoice(joinTypes) + +val leftColProjection = leftTable.columns.map(c => s"l.$c").mkString(", ") +val rightColProjection = rightTable.columns.map(c => s"r.$c").mkString(", ") +"SELECT " + + s"$leftColProjection, " + + s"$rightColProjection " + + s"FROM $leftTableName l " + + s"$joinType JOIN $rightTableName r " + + s"ON l.$leftCol = r.$rightCol " + Review Comment: Or it doesn't matter and we let Spark automatically cast between them? -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
viirya commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1622871660 ## fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala: ## @@ -0,0 +1,121 @@ +/* + * 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.comet.fuzz + +import java.io.{BufferedWriter, FileWriter} + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.sql.SparkSession + +object QueryGen { + + def generateRandomQueries( + r: Random, + spark: SparkSession, + numFiles: Int, + numQueries: Int): Unit = { +for (i <- 0 until numFiles) { + spark.read.parquet(s"test$i.parquet").createTempView(s"test$i") +} + +val w = new BufferedWriter(new FileWriter("queries.sql")) + +val uniqueQueries = mutable.HashSet[String]() + +for (_ <- 0 until numQueries) { + val sql = r.nextInt().abs % 3 match { +case 0 => generateJoin(r, spark, numFiles) +case 1 => generateAggregate(r, spark, numFiles) +case 2 => generateScalar(r, spark, numFiles) + } + if (!uniqueQueries.contains(sql)) { +uniqueQueries += sql +w.write(sql + "\n") + } +} +w.close() + } + + private def generateAggregate(r: Random, spark: SparkSession, numFiles: Int): String = { +val tableName = s"test${r.nextInt(numFiles)}" +val table = spark.table(tableName) + +val func = Utils.randomChoice(Meta.aggFunc, r) +val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + +val groupingCols = Range(0, 2).map(_ => Utils.randomChoice(table.columns, r)) + +if (groupingCols.isEmpty) { + s"SELECT ${args.mkString(", ")}, ${func.name}(${args.mkString(", ")}) AS x " + +s"FROM $tableName " + +s"ORDER BY ${args.mkString(", ")};" +} else { + s"SELECT ${groupingCols.mkString(", ")}, ${func.name}(${args.mkString(", ")}) " + +s"FROM $tableName " + +s"GROUP BY ${groupingCols.mkString(",")} " + +s"ORDER BY ${groupingCols.mkString(", ")};" +} + } + + private def generateScalar(r: Random, spark: SparkSession, numFiles: Int): String = { +val tableName = s"test${r.nextInt(numFiles)}" +val table = spark.table(tableName) + +val func = Utils.randomChoice(Meta.scalarFunc, r) +val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + +// Example SELECT c0, log(c0) as x FROM test0 +s"SELECT ${args.mkString(", ")}, ${func.name}(${args.mkString(", ")}) AS x " + + s"FROM $tableName " + + s"ORDER BY ${args.mkString(", ")};" + } + + private def generateJoin(r: Random, spark: SparkSession, numFiles: Int): String = { +val leftTableName = s"test${r.nextInt(numFiles)}" +val rightTableName = s"test${r.nextInt(numFiles)}" +val leftTable = spark.table(leftTableName) +val rightTable = spark.table(rightTableName) + +val leftCol = Utils.randomChoice(leftTable.columns, r) +val rightCol = Utils.randomChoice(rightTable.columns, r) + +val joinTypes = Seq(("INNER", 0.4), ("LEFT", 0.3), ("RIGHT", 0.3)) +val joinType = Utils.randomWeightedChoice(joinTypes) + +val leftColProjection = leftTable.columns.map(c => s"l.$c").mkString(", ") +val rightColProjection = rightTable.columns.map(c => s"r.$c").mkString(", ") +"SELECT " + + s"$leftColProjection, " + + s"$rightColProjection " + + s"FROM $leftTableName l " + + s"$joinType JOIN $rightTableName r " + + s"ON l.$leftCol = r.$rightCol " + Review Comment: `leftCol` and `rightCol` are randomly picked up. Are they always same data types? -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
viirya commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1622870392 ## fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala: ## @@ -0,0 +1,121 @@ +/* + * 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.comet.fuzz + +import java.io.{BufferedWriter, FileWriter} + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.sql.SparkSession + +object QueryGen { + + def generateRandomQueries( + r: Random, + spark: SparkSession, + numFiles: Int, + numQueries: Int): Unit = { +for (i <- 0 until numFiles) { + spark.read.parquet(s"test$i.parquet").createTempView(s"test$i") +} + +val w = new BufferedWriter(new FileWriter("queries.sql")) + +val uniqueQueries = mutable.HashSet[String]() + +for (_ <- 0 until numQueries) { + val sql = r.nextInt().abs % 3 match { +case 0 => generateJoin(r, spark, numFiles) +case 1 => generateAggregate(r, spark, numFiles) +case 2 => generateScalar(r, spark, numFiles) + } + if (!uniqueQueries.contains(sql)) { +uniqueQueries += sql +w.write(sql + "\n") + } +} +w.close() + } + + private def generateAggregate(r: Random, spark: SparkSession, numFiles: Int): String = { +val tableName = s"test${r.nextInt(numFiles)}" +val table = spark.table(tableName) + +val func = Utils.randomChoice(Meta.aggFunc, r) +val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + +val groupingCols = Range(0, 2).map(_ => Utils.randomChoice(table.columns, r)) Review Comment: Do we need to exclude gouping columns from aggregation function arguments? -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
codecov-commenter commented on PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#issuecomment-2142467971 ## [Codecov](https://app.codecov.io/gh/apache/datafusion-comet/pull/472?dropdown=coverage&src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report All modified and coverable lines are covered by tests :white_check_mark: > Project coverage is 34.15%. Comparing base [(`9125e6a`)](https://app.codecov.io/gh/apache/datafusion-comet/commit/9125e6a04dce7c86bb0e4f4f297c8c2b70a39559?dropdown=coverage&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) to head [(`fad7a52`)](https://app.codecov.io/gh/apache/datafusion-comet/commit/fad7a52b2952d0b38604f866d36c05d6515defe4?dropdown=coverage&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache). > Report is 10 commits behind head on main. Additional details and impacted files ```diff @@ Coverage Diff @@ ## main #472 +/- ## + Coverage 34.05% 34.15% +0.10% + Complexity 859 806 -53 Files 116 105 -11 Lines 3868038463 -217 Branches 8568 8554 -14 - Hits 1317113137 -34 + Misses2274622570 -176 + Partials 2763 2756 -7 ``` [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/datafusion-comet/pull/472?dropdown=coverage&src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache). :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache). -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
viirya commented on PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#issuecomment-2142413159 > I'm not sure why tests are being cancelled on this PR. I saw the same on some PRs too. Re-triggering CIs can make them back. -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove commented on PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#issuecomment-2142329103 I'm not sure why tests are being cancelled on this PR. -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove commented on PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#issuecomment-2137823739 Thanks for the review @kazuyukitanimura. I have addressed your comments and also added support for decimal type. I removed the TODO comments and added a roadmap section to the README to cover some of the planned future work. -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
kazuyukitanimura commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1617944984 ## fuzz-testing/pom.xml: ## @@ -0,0 +1,106 @@ + + +http://maven.apache.org/POM/4.0.0"; + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd";> +4.0.0 + + +org.apache.comet + comet-parent-spark${spark.version.short}_${scala.binary.version} +0.1.0-SNAPSHOT +../pom.xml + + + comet-fuzz-spark${spark.version.short}_${scala.binary.version} +comet-fuzz +http://maven.apache.org +jar + + + +false + + + + +org.scala-lang +scala-library +${scala.version} +provided + + +org.apache.spark +spark-sql_${scala.binary.version} +provided + + +org.rogach +scallop_${scala.binary.version} +5.1.0 + Review Comment: nit: (optional) we use `` for version ## fuzz-testing/src/main/scala/org/apache/comet/fuzz/DataGen.scala: ## @@ -0,0 +1,153 @@ +/* + * 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.comet.fuzz + +import java.sql.Timestamp + +import scala.util.Random + +import org.apache.spark.sql.{Row, SaveMode, SparkSession} +import org.apache.spark.sql.types.{DataType, DataTypes, StructField, StructType} + +object DataGen { + + def generateRandomFiles( + r: Random, + spark: SparkSession, + numFiles: Int, + numRows: Int, + numColumns: Int): Unit = { +for (i <- 0 until numFiles) { + generateRandomParquetFile(r, spark, s"test$i.parquet", numRows, numColumns) +} + } + + def generateRandomParquetFile( + r: Random, + spark: SparkSession, + filename: String, + numRows: Int, + numColumns: Int): Unit = { + +// TODO add examples of all supported types, including complex types +val dataTypes = Seq( + (DataTypes.ByteType, 0.2), Review Comment: What about adding boolean and binary types? -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
Re: [PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove commented on code in PR #472: URL: https://github.com/apache/datafusion-comet/pull/472#discussion_r1615306091 ## dev/scalastyle-config.xml: ## @@ -242,7 +242,7 @@ This file is divided into 3 sections: java,scala,org,apache,3rdParty,comet javax?\..* scala\..* - org\.(?!apache\.comet).* Review Comment: We should exclude all apache packages from the `org` group, not just comet. We have a separate rule below for excluding comet from the apache group. Without this change it was not possible to satisfy both scalastyle and scalafix at the same time as they wanted different import orders. -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org
[PR] feat: Add "Comet Fuzz" fuzz-testing utility [datafusion-comet]
andygrove opened a new pull request, #472: URL: https://github.com/apache/datafusion-comet/pull/472 ## Which issue does this PR close? N/A ## Rationale for this change Comet Fuzz is a standalone project for generating random data and queries and executing queries against Spark with Comet disabled and enabled and checking for incompatibilities. Although it is a simple tool it has already been useful in finding many bugs. Comet Fuzz is inspired by the [SparkFuzz](https://ir.cwi.nl/pub/30222) paper from Databricks and CWI. ## What changes are included in this PR? New standalone `comet-fuzz` project. ## How are these changes tested? Manually tested. -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org - To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org