You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/10/16 04:26:16 UTC

[GitHub] [iceberg] kbendick commented on pull request #5991: Spark: Fix DATE_ADD expression in IcebergSourceFlatParquetDataWriteBenchmark

kbendick commented on PR #5991:
URL: https://github.com/apache/iceberg/pull/5991#issuecomment-1279888693

   > Hm, not familiar with Benchmark, did this ever work for previous sparks?
   > 
   > Looks like date_add does take in int though https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L3157 so it looks good to me.
   
   I'm not sure if this was ever working, but it currently fails in master with the following error in the output file (tried 3.2).
   
   ```
   # JMH version: 1.32
   # VM version: JDK 11.0.15, OpenJDK 64-Bit Server VM, 11.0.15+9-LTS
   # VM invoker: /Library/Java/JavaVirtualMachines/amazon-corretto-11.jdk/Contents/Home/bin/java
   # VM options: -Dfile.encoding=UTF-8 -Djava.io.tmpdir=/Users/kylebendickson/repos/iceberg/spark/v3.2/spark/build/tmp/jmh -Duser.country=US -Duser.language=en -Duser.variant
   # Blackhole mode: full + dont-inline hint
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.IcebergSourceFlatParquetDataWriteBenchmark.writeIceberg
   
   # Run progress: 50.00% complete, ETA 00:00:02
   # Fork: 1 of 1
   # Warmup Iteration   1: WARNING: An illegal reflective access operation has occurred
   WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/kylebendickson/repos/iceberg/spark/v3.2/spark/build/libs/iceberg-spark-3.2_2.12-0.15.0-SNAPSHOT-jmh.jar) to constructor java.nio.DirectByteBuffer(long,int)
   WARNING: Please consider reporting this to the maintainers of org.apache.spark.unsafe.Platform
   WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations
   WARNING: All illegal access operations will be denied in a future release
   <failure>
   
   org.apache.spark.sql.AnalysisException: cannot resolve 'date_add(current_date(), (longCol % CAST(20 AS BIGINT)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '(longCol % CAST(20 AS BIGINT))' is of bigint type.; line 1 pos 0;
   'Project [longCol#2L, intCol#4, floatCol#7, doubleCol#11, decimalCol#16, date_add(current_date(Some(America/Los_Angeles)), (longCol#2L % cast(20 as bigint))) AS dateCol#22]
   +- Project [longCol#2L, intCol#4, floatCol#7, doubleCol#11, cast(longCol#2L as decimal(20,5)) AS decimalCol#16]
      +- Project [longCol#2L, intCol#4, floatCol#7, cast(longCol#2L as double) AS doubleCol#11]
         +- Project [longCol#2L, intCol#4, cast(longCol#2L as float) AS floatCol#7]
            +- Project [longCol#2L, cast(longCol#2L as int) AS intCol#4]
               +- Project [id#0L AS longCol#2L]
                  +- Range (0, 5000000, step=1, splits=Some(1))
   
           at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
           at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:193)
           at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$$nestedInanonfun$checkAnalysis$1$2.applyOrElse(CheckAnalysis.scala:178)
           at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUpWithPruning$2(TreeNode.scala:535)
           at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82)
           at org.apache.spark.sql.catalyst.trees.TreeNode.transformUpWithPruning(TreeNode.scala:535)
           at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUpWithPruning$1(TreeNode.scala:532)
           at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1122)
           at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1121)
           at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:467)
           at org.apache.spark.sql.catalyst.trees.TreeNode.transformUpWithPruning(TreeNode.scala:532)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$transformExpressionsUpWithPruning$1(QueryPlan.scala:181)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:193)
           at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:193)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:204)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$3(QueryPlan.scala:209)
           at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
           at scala.collection.immutable.List.foreach(List.scala:431)
           at scala.collection.TraversableLike.map(TraversableLike.scala:286)
           at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
           at scala.collection.immutable.List.map(List.scala:305)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:209)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:214)
           at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:323)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:214)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsUpWithPruning(QueryPlan.scala:181)
           at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsUp(QueryPlan.scala:161)
           at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$1(CheckAnalysis.scala:178)
           at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$1$adapted(CheckAnalysis.scala:97)
           at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:263)
           at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis(CheckAnalysis.scala:97)
           at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis$(CheckAnalysis.scala:92)
           at org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:182)
           at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:205)
           at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:330)
           at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:202)
           at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:75)
           at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
           at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:183)
           at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
           at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:183)
           at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:75)
           at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:73)
           at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:65)
           at org.apache.spark.sql.Dataset$.$anonfun$ofRows$1(Dataset.scala:90)
           at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
           at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:88)
           at org.apache.spark.sql.Dataset.withPlan(Dataset.scala:3734)
           at org.apache.spark.sql.Dataset.select(Dataset.scala:1454)
           at org.apache.spark.sql.Dataset.withColumns(Dataset.scala:2417)
           at org.apache.spark.sql.Dataset.withColumn(Dataset.scala:2384)
           at org.apache.iceberg.spark.source.parquet.IcebergSourceFlatParquetDataWriteBenchmark.benchmarkData(IcebergSourceFlatParquetDataWriteBenchmark.java:84)
           at org.apache.iceberg.spark.source.parquet.IcebergSourceFlatParquetDataWriteBenchmark.writeIceberg(IcebergSourceFlatParquetDataWriteBenchmark.java:65)
           at org.apache.iceberg.spark.source.parquet.jmh_generated.IcebergSourceFlatParquetDataWriteBenchmark_writeIceberg_jmhTest.writeIceberg_ss_jmhStub(IcebergSourceFlatParquetDataWriteBenchmark_writeIceberg_jmhTest.java:416)
           at org.apache.iceberg.spark.source.parquet.jmh_generated.IcebergSourceFlatParquetDataWriteBenchmark_writeIceberg_jmhTest.writeIceberg_SingleShotTime(IcebergSourceFlatParquetDataWriteBenchmark_writeIceberg_jmhTest.java:371)
           at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.base/java.lang.reflect.Method.invoke(Method.java:566)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:470)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453)
           at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
           at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
           at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
           at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
           at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
           at java.base/java.lang.Thread.run(Thread.java:829)
   
   
   
   
   # Run complete. Total time: 00:00:06
   
   REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on
   why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial
   experiments, perform baseline and negative tests that provide experimental control, make sure
   the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts.
   Do not assume the numbers tell you what you want them to tell.
   
   Benchmark  Mode  Cnt  Score   Error  Units
   
   Benchmark result is saved to /Users/kylebendickson/repos/iceberg/spark/v3.2/spark/build/results/jmh/results.txt
   ```
   
   This patch resolves the issue though (at least on 3.2).
   
   Thanks @dramaticlly!
   


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org