Shardul Mahadik created SPARK-38510: ---------------------------------------
Summary: Failure fetching JSON representation of Spark plans with Hive UDFs Key: SPARK-38510 URL: https://issues.apache.org/jira/browse/SPARK-38510 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.3.0 Reporter: Shardul Mahadik Repro: {code:java} scala> spark.sql("CREATE TEMPORARY FUNCTION test_udf AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFAesEncrypt'") scala> spark.sql("SELECT test_udf('a', 'b')").queryExecution.analyzed.toJSON scala.reflect.internal.Symbols$CyclicReference: illegal cyclic reference involving class InterfaceAudience java.lang.RuntimeException: error reading Scala signature of org.apache.spark.sql.hive.HiveGenericUDF: illegal cyclic reference involving class InterfaceAudience at scala.reflect.internal.pickling.UnPickler.unpickle(UnPickler.scala:51) at scala.reflect.runtime.JavaMirrors$JavaMirror.unpickleClass(JavaMirrors.scala:660) at scala.reflect.runtime.SymbolLoaders$TopClassCompleter.$anonfun$complete$2(SymbolLoaders.scala:37) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at scala.reflect.internal.SymbolTable.slowButSafeEnteringPhaseNotLaterThan(SymbolTable.scala:333) at scala.reflect.runtime.SymbolLoaders$TopClassCompleter.complete(SymbolLoaders.scala:34) at scala.reflect.internal.Symbols$Symbol.completeInfo(Symbols.scala:1551) at scala.reflect.internal.Symbols$Symbol.info(Symbols.scala:1514) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$7.scala$reflect$runtime$SynchronizedSymbols$SynchronizedSymbol$$super$info(SynchronizedSymbols.scala:203) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol.$anonfun$info$1(SynchronizedSymbols.scala:158) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol.info(SynchronizedSymbols.scala:149) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol.info$(SynchronizedSymbols.scala:158) at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anon$7.info(SynchronizedSymbols.scala:203) at scala.reflect.internal.Symbols$Symbol.initialize(Symbols.scala:1698) at scala.reflect.internal.Symbols$SymbolContextApiImpl.selfType(Symbols.scala:151) at scala.reflect.internal.Symbols$ClassSymbol.selfType(Symbols.scala:3287) at org.apache.spark.sql.catalyst.ScalaReflection$.getConstructorParameterNames(ScalaReflection.scala:656) at org.apache.spark.sql.catalyst.trees.TreeNode.jsonFields(TreeNode.scala:1019) at org.apache.spark.sql.catalyst.trees.TreeNode.collectJsonValue$1(TreeNode.scala:1009) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$jsonValue$1(TreeNode.scala:1011) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$jsonValue$1$adapted(TreeNode.scala:1011) at scala.collection.Iterator.foreach(Iterator.scala:943) at scala.collection.Iterator.foreach$(Iterator.scala:943) at scala.collection.AbstractIterator.foreach(Iterator.scala:1431) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at org.apache.spark.sql.catalyst.trees.TreeNode.collectJsonValue$1(TreeNode.scala:1011) at org.apache.spark.sql.catalyst.trees.TreeNode.jsonValue(TreeNode.scala:1014) at org.apache.spark.sql.catalyst.trees.TreeNode.parseToJson(TreeNode.scala:1057) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$parseToJson$11(TreeNode.scala:1063) at scala.collection.immutable.List.map(List.scala:293) at org.apache.spark.sql.catalyst.trees.TreeNode.parseToJson(TreeNode.scala:1063) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$jsonFields$2(TreeNode.scala:1033) at scala.collection.immutable.List.map(List.scala:293) at org.apache.spark.sql.catalyst.trees.TreeNode.jsonFields(TreeNode.scala:1024) at org.apache.spark.sql.catalyst.trees.TreeNode.collectJsonValue$1(TreeNode.scala:1009) at org.apache.spark.sql.catalyst.trees.TreeNode.jsonValue(TreeNode.scala:1014) at org.apache.spark.sql.catalyst.trees.TreeNode.toJSON(TreeNode.scala:1000) ... 47 elided {code} This issue is due to [bug#12190 in Scala|https://github.com/scala/bug/issues/12190] which does not handle cyclic references in Java annotations correctly. The cyclic reference in this case comes from InterfaceAudience annotation which [annotates itself|https://github.com/apache/hadoop/blob/db8ae4b65448c506c9234641b2c1f9b8e894dc18/hadoop-common-project/hadoop-annotations/src/main/java/org/apache/hadoop/classification/InterfaceAudience.java#L45]. This annotation class is present in the type hierarchy of {{{}HiveGenericUDF{}}}. A simple workaround for this issue, is to just retry the operation. It will succeed on the retry probably because the annotation is partially resolved from the previous attempt. -- This message was sent by Atlassian Jira (v8.20.1#820001) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org