-
Notifications
You must be signed in to change notification settings - Fork 907
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
lineage plugin throws java.util.NoSuchElementException: None.get #6384
Comments
Can you provide sql to reproduce? |
df1 = spark.readStream query = df.writeStream.queryName("ads-work-real-starrocks") def process_row(df): |
这种场景也会出现警告信息,能否这个警告信息配置在后台日志或者去掉
|
@wForget 现在不管是命令行方式执行还是后台spark-submit 都会出现这个警告信息,特别是实时任务,日志频繁警告,能否有什么方式处理下该警告 |
From error stack, it seems that Lines 491 to 495 in 86ce5e1
|
If it is difficult to determine which datasource table caused the exception, we can simply handle it like this:
|
scala> val workDF = spark.read.format("starrocks").option("starrocks.table.identifier", "ads.ads_real_work_sr").option("starrocks.fe.http.url", urlHttp).option("starrocks.fe.jdbc.url", urlJdbc).option("starrocks.user", userName).option("starrocks.password", password).load() scala> workDF.createOrReplaceTempView("temp_work_240511") scala> val execSql = "select a.source_id,b.main_id from ods.ods_biz_aftersale_refund_base a left join (select id,main_id from temp_work_240511 where nvl(region,0) = 202405) b on a.source_id = b.id where b.main_id isnot null" scala> val resDF = spark.sql(execSql) scala> resDF.show() scala> 24/05/11 18:59:37 WARN SparkSQLLineageParseHelper: Extract Statement[8] columns lineage failed. |
private def getV2TableName(plan: NamedRelation): String = { |
Can you provide error details? |
已修改代码 以兼容异常警告 |
Can you send a PR for this fix? |
Code of Conduct
Search before asking
Describe the bug
使用kyuubi-spark-lineage模块做sparksql的数据血缘到atlas(参考https://kyuubi.readthedocs.io/en/v1.8.1-docs/extensions/engines/spark/lineage.html#get-lineage-events),一直出现该异常信息。
24/05/10 11:32:19 WARN SparkSQLLineageParseHelper: Extract Statement[30918] columns lineage failed.
java.util.NoSuchElementException: None.get
at scala.None$.get(Option.scala:529) ~[scala-library-2.12.15.jar:?]
at scala.None$.get(Option.scala:527) ~[scala-library-2.12.15.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.LineageParser.getV2TableName(SparkSQLLineageParseHelper.scala:493) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.LineageParser.extractColumnsLineage(SparkSQLLineageParseHelper.scala:304) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.LineageParser.parse(SparkSQLLineageParseHelper.scala:54) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.LineageParser.parse$(SparkSQLLineageParseHelper.scala:52) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.SparkSQLLineageParseHelper.parse(SparkSQLLineageParseHelper.scala:510) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.SparkSQLLineageParseHelper.$anonfun$transformToLineage$1(SparkSQLLineageParseHelper.scala:516) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at scala.util.Try$.apply(Try.scala:213) ~[scala-library-2.12.15.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.SparkSQLLineageParseHelper.transformToLineage(SparkSQLLineageParseHelper.scala:516) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.SparkOperationLineageQueryExecutionListener.onSuccess(SparkOperationLineageQueryExecutionListener.scala:34) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.spark.sql.util.ExecutionListenerBus.doPostEvent(QueryExecutionListener.scala:165) ~[spark-sql_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.sql.util.ExecutionListenerBus.doPostEvent(QueryExecutionListener.scala:135) ~[spark-sql_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.ListenerBus.postToAll(ListenerBus.scala:117) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.ListenerBus.postToAll$(ListenerBus.scala:101) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.sql.util.ExecutionListenerBus.postToAll(QueryExecutionListener.scala:135) ~[spark-sql_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.sql.util.ExecutionListenerBus.onOtherEvent(QueryExecutionListener.scala:147) ~[spark-sql_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.SparkListenerBus.doPostEvent(SparkListenerBus.scala:100) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.SparkListenerBus.doPostEvent$(SparkListenerBus.scala:28) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.ListenerBus.postToAll(ListenerBus.scala:117) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.ListenerBus.postToAll$(ListenerBus.scala:101) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue.super$postToAll(AsyncEventQueue.scala:105) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue.$anonfun$dispatch$1(AsyncEventQueue.scala:105) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at scala.runtime.java8.JFunction0$mcJ$sp.apply(JFunction0$mcJ$sp.java:23) ~[scala-library-2.12.15.jar:?]
at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62) ~[scala-library-2.12.15.jar:?]
at org.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:100) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue$$anon$2.$anonfun$run$1(AsyncEventQueue.scala:96) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1446) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue$$anon$2.run(AsyncEventQueue.scala:96) ~[spark-core_2.12-3.3.1.jar:3.3.1]
Affects Version(s)
kyuubi-spark-lineage 1.8.1
Kyuubi Server Log Output
No response
Kyuubi Engine Log Output
No response
Kyuubi Server Configurations
No response
Kyuubi Engine Configurations
No response
Additional context
24/05/10 11:32:19 WARN SparkSQLLineageParseHelper: Extract Statement[30918] columns lineage failed.
java.util.NoSuchElementException: None.get
at scala.None$.get(Option.scala:529) ~[scala-library-2.12.15.jar:?]
at scala.None$.get(Option.scala:527) ~[scala-library-2.12.15.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.LineageParser.getV2TableName(SparkSQLLineageParseHelper.scala:493) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.LineageParser.extractColumnsLineage(SparkSQLLineageParseHelper.scala:304) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.LineageParser.parse(SparkSQLLineageParseHelper.scala:54) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.LineageParser.parse$(SparkSQLLineageParseHelper.scala:52) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.SparkSQLLineageParseHelper.parse(SparkSQLLineageParseHelper.scala:510) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.SparkSQLLineageParseHelper.$anonfun$transformToLineage$1(SparkSQLLineageParseHelper.scala:516) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at scala.util.Try$.apply(Try.scala:213) ~[scala-library-2.12.15.jar:?]
at org.apache.kyuubi.plugin.lineage.helper.SparkSQLLineageParseHelper.transformToLineage(SparkSQLLineageParseHelper.scala:516) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.kyuubi.plugin.lineage.SparkOperationLineageQueryExecutionListener.onSuccess(SparkOperationLineageQueryExecutionListener.scala:34) ~[kyuubi-spark-lineage_2.12-1.8.1-jar-with-dependencies.jar:?]
at org.apache.spark.sql.util.ExecutionListenerBus.doPostEvent(QueryExecutionListener.scala:165) ~[spark-sql_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.sql.util.ExecutionListenerBus.doPostEvent(QueryExecutionListener.scala:135) ~[spark-sql_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.ListenerBus.postToAll(ListenerBus.scala:117) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.ListenerBus.postToAll$(ListenerBus.scala:101) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.sql.util.ExecutionListenerBus.postToAll(QueryExecutionListener.scala:135) ~[spark-sql_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.sql.util.ExecutionListenerBus.onOtherEvent(QueryExecutionListener.scala:147) ~[spark-sql_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.SparkListenerBus.doPostEvent(SparkListenerBus.scala:100) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.SparkListenerBus.doPostEvent$(SparkListenerBus.scala:28) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.ListenerBus.postToAll(ListenerBus.scala:117) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.ListenerBus.postToAll$(ListenerBus.scala:101) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue.super$postToAll(AsyncEventQueue.scala:105) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue.$anonfun$dispatch$1(AsyncEventQueue.scala:105) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at scala.runtime.java8.JFunction0$mcJ$sp.apply(JFunction0$mcJ$sp.java:23) ~[scala-library-2.12.15.jar:?]
at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62) ~[scala-library-2.12.15.jar:?]
at org.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:100) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue$$anon$2.$anonfun$run$1(AsyncEventQueue.scala:96) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1446) ~[spark-core_2.12-3.3.1.jar:3.3.1]
at org.apache.spark.scheduler.AsyncEventQueue$$anon$2.run(AsyncEventQueue.scala:96) ~[spark-core_2.12-3.3.1.jar:3.3.1]
Are you willing to submit PR?
The text was updated successfully, but these errors were encountered: