@@ -24,13 +24,15 @@ import java.io.File
2424import scala .collection .JavaConverters ._
2525import scala .util .Random
2626
27+ import org .apache .hadoop .fs .Path
2728import org .apache .spark .TestUtils
2829import org .apache .spark .benchmark .Benchmark
30+ import org .apache .spark .sql .{DataFrame , SparkSession }
2931import org .apache .spark .sql .execution .datasources .parquet .VectorizedParquetRecordReader
3032import org .apache .spark .sql .types ._
3133import org .apache .spark .sql .vectorized .ColumnVector
3234
33- import org .apache .comet .CometConf
35+ import org .apache .comet .{ CometConf , WithHdfsCluster }
3436import org .apache .comet .CometConf .{SCAN_NATIVE_COMET , SCAN_NATIVE_DATAFUSION , SCAN_NATIVE_ICEBERG_COMPAT }
3537import org .apache .comet .parquet .BatchReader
3638
@@ -40,7 +42,7 @@ import org.apache.comet.parquet.BatchReader
4042 * benchmark-org.apache.spark.sql.benchmark.CometReadBenchmark` Results will be written to
4143 * "spark/benchmarks/CometReadBenchmark-**results.txt".
4244 */
43- object CometReadBenchmark extends CometBenchmarkBase {
45+ class CometReadBaseBenchmark extends CometBenchmarkBase {
4446
4547 def numericScanBenchmark (values : Int , dataType : DataType ): Unit = {
4648 // Benchmarks running through spark sql.
@@ -71,6 +73,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
7173 sqlBenchmark.addCase(" SQL Parquet - Comet Native DataFusion" ) { _ =>
7274 withSQLConf(
7375 CometConf .COMET_ENABLED .key -> " true" ,
76+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
7477 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_DATAFUSION ) {
7578 spark.sql(s " select $query from parquetV1Table " ).noop()
7679 }
@@ -79,6 +82,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
7982 sqlBenchmark.addCase(" SQL Parquet - Comet Native Iceberg Compat" ) { _ =>
8083 withSQLConf(
8184 CometConf .COMET_ENABLED .key -> " true" ,
85+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
8286 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_ICEBERG_COMPAT ) {
8387 spark.sql(s " select $query from parquetV1Table " ).noop()
8488 }
@@ -118,6 +122,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
118122 sqlBenchmark.addCase(" SQL Parquet - Comet Native DataFusion" ) { _ =>
119123 withSQLConf(
120124 CometConf .COMET_ENABLED .key -> " true" ,
125+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
121126 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_DATAFUSION ) {
122127 spark.sql(" select sum(id) from parquetV1Table" ).noop()
123128 }
@@ -126,6 +131,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
126131 sqlBenchmark.addCase(" SQL Parquet - Comet Native Iceberg Compat" ) { _ =>
127132 withSQLConf(
128133 CometConf .COMET_ENABLED .key -> " true" ,
134+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
129135 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_ICEBERG_COMPAT ) {
130136 spark.sql(" select sum(id) from parquetV1Table" ).noop()
131137 }
@@ -244,6 +250,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
244250 benchmark.addCase(" SQL Parquet - Comet Native DataFusion" ) { _ =>
245251 withSQLConf(
246252 CometConf .COMET_ENABLED .key -> " true" ,
253+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
247254 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_DATAFUSION ) {
248255 spark.sql(" select sum(c2) from parquetV1Table where c1 + 1 > 0" ).noop()
249256 }
@@ -252,6 +259,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
252259 benchmark.addCase(" SQL Parquet - Comet Native Iceberg Compat" ) { _ =>
253260 withSQLConf(
254261 CometConf .COMET_ENABLED .key -> " true" ,
262+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
255263 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_ICEBERG_COMPAT ) {
256264 spark.sql(" select sum(c2) from parquetV1Table where c1 + 1 > 0" ).noop()
257265 }
@@ -300,6 +308,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
300308 sqlBenchmark.addCase(" SQL Parquet - Comet Native DataFusion" ) { _ =>
301309 withSQLConf(
302310 CometConf .COMET_ENABLED .key -> " true" ,
311+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
303312 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_DATAFUSION ) {
304313 spark.sql(" select sum(length(id)) from parquetV1Table" ).noop()
305314 }
@@ -308,6 +317,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
308317 sqlBenchmark.addCase(" SQL Parquet - Comet Native Iceberg Compat" ) { _ =>
309318 withSQLConf(
310319 CometConf .COMET_ENABLED .key -> " true" ,
320+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
311321 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_ICEBERG_COMPAT ) {
312322 spark.sql(" select sum(length(id)) from parquetV1Table" ).noop()
313323 }
@@ -352,6 +362,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
352362 benchmark.addCase(" SQL Parquet - Comet Native DataFusion" ) { _ =>
353363 withSQLConf(
354364 CometConf .COMET_ENABLED .key -> " true" ,
365+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
355366 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_DATAFUSION ) {
356367 spark
357368 .sql(" select sum(length(c2)) from parquetV1Table where c1 is " +
@@ -363,6 +374,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
363374 benchmark.addCase(" SQL Parquet - Comet Native Iceberg Compat" ) { _ =>
364375 withSQLConf(
365376 CometConf .COMET_ENABLED .key -> " true" ,
377+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
366378 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_ICEBERG_COMPAT ) {
367379 spark
368380 .sql(" select sum(length(c2)) from parquetV1Table where c1 is " +
@@ -403,6 +415,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
403415 benchmark.addCase(" SQL Parquet - Comet Native DataFusion" ) { _ =>
404416 withSQLConf(
405417 CometConf .COMET_ENABLED .key -> " true" ,
418+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
406419 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_DATAFUSION ) {
407420 spark.sql(s " SELECT sum(c $middle) FROM parquetV1Table " ).noop()
408421 }
@@ -411,6 +424,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
411424 benchmark.addCase(" SQL Parquet - Comet Native Iceberg Compat" ) { _ =>
412425 withSQLConf(
413426 CometConf .COMET_ENABLED .key -> " true" ,
427+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
414428 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_ICEBERG_COMPAT ) {
415429 spark.sql(s " SELECT sum(c $middle) FROM parquetV1Table " ).noop()
416430 }
@@ -452,6 +466,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
452466 benchmark.addCase(" SQL Parquet - Comet Native DataFusion" ) { _ =>
453467 withSQLConf(
454468 CometConf .COMET_ENABLED .key -> " true" ,
469+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
455470 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_DATAFUSION ) {
456471 spark.sql(" SELECT * FROM parquetV1Table WHERE c1 + 1 > 0" ).noop()
457472 }
@@ -460,6 +475,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
460475 benchmark.addCase(" SQL Parquet - Comet Native Iceberg Compat" ) { _ =>
461476 withSQLConf(
462477 CometConf .COMET_ENABLED .key -> " true" ,
478+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
463479 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_ICEBERG_COMPAT ) {
464480 spark.sql(" SELECT * FROM parquetV1Table WHERE c1 + 1 > 0" ).noop()
465481 }
@@ -501,6 +517,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
501517 benchmark.addCase(" SQL Parquet - Comet Native DataFusion" ) { _ =>
502518 withSQLConf(
503519 CometConf .COMET_ENABLED .key -> " true" ,
520+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
504521 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_DATAFUSION ) {
505522 spark.sql(" SELECT * FROM parquetV1Table WHERE c1 + 1 > 0" ).noop()
506523 }
@@ -509,6 +526,7 @@ object CometReadBenchmark extends CometBenchmarkBase {
509526 benchmark.addCase(" SQL Parquet - Comet Native Iceberg Compat" ) { _ =>
510527 withSQLConf(
511528 CometConf .COMET_ENABLED .key -> " true" ,
529+ CometConf .COMET_EXEC_ENABLED .key -> " true" ,
512530 CometConf .COMET_NATIVE_SCAN_IMPL .key -> SCAN_NATIVE_ICEBERG_COMPAT ) {
513531 spark.sql(" SELECT * FROM parquetV1Table WHERE c1 + 1 > 0" ).noop()
514532 }
@@ -587,3 +605,51 @@ object CometReadBenchmark extends CometBenchmarkBase {
587605 }
588606 }
589607}
608+
609+ object CometReadBenchmark extends CometReadBaseBenchmark {}
610+
611+ object CometReadHdfsBenchmark extends CometReadBaseBenchmark with WithHdfsCluster {
612+
613+ override def getSparkSession : SparkSession = {
614+ // start HDFS cluster and add hadoop conf
615+ startHdfsCluster()
616+ val sparkSession = super .getSparkSession
617+ sparkSession.sparkContext.hadoopConfiguration.addResource(getHadoopConfFile)
618+ sparkSession
619+ }
620+
621+ override def runCometBenchmark (mainArgs : Array [String ]): Unit = {
622+ try {
623+ super .runCometBenchmark(mainArgs)
624+ } finally {
625+ stopHdfsCluster()
626+ }
627+ }
628+
629+ override def readerBenchmark (values : Int , dataType : DataType ): Unit = {
630+ // ignore reader benchmark for HDFS
631+ }
632+
633+ // mock local dir to hdfs
634+ override protected def withTempPath (f : File => Unit ): Unit = {
635+ super .withTempPath { dir =>
636+ val tempHdfsPath = new Path (getTmpRootDir, dir.getName)
637+ getFileSystem.mkdirs(tempHdfsPath)
638+ try f(dir)
639+ finally getFileSystem.delete(tempHdfsPath, true )
640+ }
641+ }
642+ override protected def prepareTable (
643+ dir : File ,
644+ df : DataFrame ,
645+ partition : Option [String ]): Unit = {
646+ val testDf = if (partition.isDefined) {
647+ df.write.partitionBy(partition.get)
648+ } else {
649+ df.write
650+ }
651+ val tempHdfsPath = getFileSystem.resolvePath(new Path (getTmpRootDir, dir.getName))
652+ val parquetV1Path = new Path (tempHdfsPath, " parquetV1" )
653+ saveAsParquetV1Table(testDf, parquetV1Path.toString)
654+ }
655+ }
0 commit comments