It looks like delta calls org.apache.spark.Utils, which is technically a private class in Spark. The signature of Utils.classForName changed (in the bytecode) to take two more params. Either delta would have to cross-compile for Spark 2 vs 3, or, needs to avoid calling Utils, or we can add a small change in Spark to keep the old Utils.classForName signature in the bytecode. However, I don't know if the last option would just reveal further dependencies on stuff that changed in Spark 3. This is worth raising on the Delta OSS project too for an opinion.
On Mon, Dec 30, 2019 at 7:52 AM Jean-Georges Perrin <j...@jgp.net> wrote: > > Hi there, > > Trying to run a very simple app saving content of a dataframe to Delta Lake. > Code works great on 2.4.4 but fails on 3.0.0 preview & preview 2. Tried on > both Delta Lake 0.5.0 and 0.4.0. > > Code (I know, it’s amazing): > > df.write().format("delta") > .mode("overwrite") > .save("/tmp/delta_grand_debat_events"); > > Exception raised: > > Exception in thread "main" com.google.common.util.concurrent.ExecutionError: > java.lang.NoSuchMethodError: > org/apache/spark/util/Utils$.classForName(Ljava/lang/String;)Ljava/lang/Class; > (loaded from > file:/Users/jgp/.m2/repository/org/apache/spark/spark-core_2.12/3.0.0-preview/spark-core_2.12-3.0.0-preview.jar > by sun.misc.Launcher$AppClassLoader@7da46134) called from interface > org.apache.spark.sql.delta.storage.LogStoreProvider (loaded from > file:/Users/jgp/.m2/repository/io/delta/delta-core_2.12/0.5.0/delta-core_2.12-0.5.0.jar > by sun.misc.Launcher$AppClassLoader@7da46134). > at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2199) > at com.google.common.cache.LocalCache.get(LocalCache.java:3934) > at > com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4736) > at org.apache.spark.sql.delta.DeltaLog$.apply(DeltaLog.scala:740) > at org.apache.spark.sql.delta.DeltaLog$.forTable(DeltaLog.scala:702) > at > org.apache.spark.sql.delta.sources.DeltaDataSource.createRelation(DeltaDataSource.scala:126) > at > org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:46) > at > org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:71) > at > org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:69) > at > org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:87) > at > org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:189) > at > org.apache.spark.sql.execution.SparkPlan$$Lambda$1437.00000000CC2CD020.apply(Unknown > Source) > at > org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:227) > at > org.apache.spark.sql.execution.SparkPlan$$Lambda$1461.00000000CFA16C20.apply(Unknown > Source) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:224) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:185) > at > org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:110) > at > org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:109) > at > org.apache.spark.sql.DataFrameWriter.$anonfun$runCommand$1(DataFrameWriter.scala:829) > at > org.apache.spark.sql.DataFrameWriter$$Lambda$2070.00000000CFB38020.apply(Unknown > Source) > at > org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$4(SQLExecution.scala:100) > at > org.apache.spark.sql.execution.SQLExecution$$$Lambda$1155.00000000CF955820.apply(Unknown > Source) > at > org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:87) > at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:829) > at > org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:309) > at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:293) > at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:236) > at > net.jgp.books.spark.ch17.lab200_feed_delta.FeedDeltaLakeApp.start(FeedDeltaLakeApp.java:131) > at > net.jgp.books.spark.ch17.lab200_feed_delta.FeedDeltaLakeApp.main(FeedDeltaLakeApp.java:29) > Caused by: java.lang.NoSuchMethodError: > org/apache/spark/util/Utils$.classForName(Ljava/lang/String;)Ljava/lang/Class; > (loaded from > file:/Users/jgp/.m2/repository/org/apache/spark/spark-core_2.12/3.0.0-preview/spark-core_2.12-3.0.0-preview.jar > by sun.misc.Launcher$AppClassLoader@7da46134) called from interface > org.apache.spark.sql.delta.storage.LogStoreProvider (loaded from > file:/Users/jgp/.m2/repository/io/delta/delta-core_2.12/0.5.0/delta-core_2.12-0.5.0.jar > by sun.misc.Launcher$AppClassLoader@7da46134). > at > org.apache.spark.sql.delta.storage.LogStoreProvider.createLogStore(LogStore.scala:122) > at > org.apache.spark.sql.delta.storage.LogStoreProvider.createLogStore$(LogStore.scala:120) > at org.apache.spark.sql.delta.DeltaLog.createLogStore(DeltaLog.scala:58) > at > org.apache.spark.sql.delta.storage.LogStoreProvider.createLogStore(LogStore.scala:117) > at > org.apache.spark.sql.delta.storage.LogStoreProvider.createLogStore$(LogStore.scala:115) > at org.apache.spark.sql.delta.DeltaLog.createLogStore(DeltaLog.scala:58) > at org.apache.spark.sql.delta.DeltaLog.<init>(DeltaLog.scala:79) > at > org.apache.spark.sql.delta.DeltaLog$$anon$3.$anonfun$call$2(DeltaLog.scala:744) > at > org.apache.spark.sql.delta.DeltaLog$$anon$3$$Lambda$2122.00000000CBBD5C20.apply(Unknown > Source) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194) > at > org.apache.spark.sql.delta.DeltaLog$$anon$3.$anonfun$call$1(DeltaLog.scala:744) > at > org.apache.spark.sql.delta.DeltaLog$$anon$3$$Lambda$2121.00000000CB4AA420.apply(Unknown > Source) > at > com.databricks.spark.util.DatabricksLogging.recordOperation(DatabricksLogging.scala:77) > at > com.databricks.spark.util.DatabricksLogging.recordOperation$(DatabricksLogging.scala:67) > at org.apache.spark.sql.delta.DeltaLog$.recordOperation(DeltaLog.scala:671) > at > org.apache.spark.sql.delta.metering.DeltaLogging.recordDeltaOperation(DeltaLogging.scala:103) > at > org.apache.spark.sql.delta.metering.DeltaLogging.recordDeltaOperation$(DeltaLogging.scala:89) > at > org.apache.spark.sql.delta.DeltaLog$.recordDeltaOperation(DeltaLog.scala:671) > at org.apache.spark.sql.delta.DeltaLog$$anon$3.call(DeltaLog.scala:743) > at org.apache.spark.sql.delta.DeltaLog$$anon$3.call(DeltaLog.scala:740) > at > com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4739) > at > com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3524) > at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2317) > at > com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2280) > at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2195) > ... 30 more > > Any clue of something I am doing wrong? > > jg > > > --------------------------------------------------------------------- To unsubscribe e-mail: dev-unsubscr...@spark.apache.org