-
Notifications
You must be signed in to change notification settings - Fork 28k
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
[SPARK-47289][SQL] Allow extensions to log extended information in explain plan #45488
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi, @parthchandra .
Please enable GitHub Action in your repository.
@dongjoon-hyun github actions are enabled in my repository and the branch is based on the latest commit in master. In my repo the ci checks are shown as passing. |
db309c7
to
94d1b40
Compare
Rebased on latest master and tests are passing now |
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you for updating. +1, LGTM (Pending CIs).
cc @cloud-fan , @HyukjinKwon , too
@@ -583,6 +611,24 @@ case class MyParser(spark: SparkSession, delegate: ParserInterface) extends Pars | |||
delegate.parseQuery(sqlText) | |||
} | |||
|
|||
case class ExplainPlanInfoTagRule() extends Rule[SparkPlan] { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is really a hacky way to provide extended EXPLAIN info. Can we add a interface like
trait ExtendedExplainGenerator {
def generateExtendedInfo(plan: SparkPlan): String
}
Then the config provides the implementation class.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let me see if I understand this right -
A session extension implementation can implement the trait above and set the implementation in a config. If such an implementation is provided, then explain plan can print it. We can replace the enableExtensionInfo
flag with this new conf.
The contents and formatting of the extended explain string etc, now become part of the session extension and so we can remove the changes in QueryPlan
. The extension implementors will have to do more work, but likely will have greater flexibility.
Let me know if I misunderstood. Otherwise I'll go ahead with this change.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you for the review, @cloud-fan .
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, the EXPLAIN command will call this plugin implementation and append the result to the command output as extended information. Then we don't need to add new APIs in QueryPlan
and the plugin can also be more flexible as it sees the full plan tree.
CI test failure appears to be unrelated. Not re-running, because this PR will change. |
@cloud-fan I've update this PR. Can you please take a look? |
import scala.io.Source | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit. Could you run dev/lint-scala
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
* A trait for a session extension to implement that provides addition explain plan | ||
* information. | ||
*/ | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit. Remove empty line.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
removed
try { | ||
val extensionClass = Utils.classForName(generator) | ||
val extension = extensionClass.getConstructor().newInstance() | ||
.asInstanceOf[ExtendedExplainGenerator] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just a question. Do you know how many times we need to generate?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think it gets called too often. It should only get called in explain plan or when the plan is displayed in the UI.
I can make this a lazy val and load it just once in the class if you think that is preferable.
@@ -333,6 +333,13 @@ object SQLConf { | |||
.booleanConf | |||
.createWithDefault(true) | |||
|
|||
val EXTENDED_EXPLAIN_PROVIDER = buildConf("spark.sql.extendedExplainProvider") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
StaticSQLConf
might be a better place for this conf. We don't want to change this per query, right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Moved to static conf
* A trait for a session extension to implement that provides addition explain plan | ||
* information. | ||
*/ | ||
trait ExtendedExplainGenerator { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
let's add
@DeveloperApi
@Since("4.0.0")
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point. Thanks!
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.sql.execution |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is an internal package. Let's put it in org.apache.spark.sql
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
@@ -369,6 +372,26 @@ class QueryExecution( | |||
Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, message) | |||
} | |||
|
|||
def extendedExplainInfo(append: String => Unit, plan: SparkPlan): Unit = { | |||
try { | |||
val generator = sparkSession.sparkContext.conf.get(EXTENDED_EXPLAIN_PROVIDER.key) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
looking at the implementation, seems the config can be a session config instead of static?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It was originally a session config, then after a previous review comment, made it static. Thinking about it again, it seems it might be better to allow it as a session conf. The extended info can then be turned on/off for specific sessions among other cases.
try { | ||
val extensionClass = Utils.classForName(generator) | ||
val extension = extensionClass.getConstructor().newInstance() | ||
.asInstanceOf[ExtendedExplainGenerator] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's call Utils.loadExtensions
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good suggestion. I was not aware of this method. It is much cleaner to use.
Ironically, we do not use this method to load session extensions!
} catch { | ||
case e@(_: ClassCastException | | ||
_: ClassNotFoundException | | ||
_: NoClassDefFoundError) => |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about catch NonFatal
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
generators, | ||
sparkSession.sparkContext.conf) | ||
if (extensions.nonEmpty) { | ||
append("\n== Extended Information ==\n") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since we support more than one ExtendedExplainGenerator
, shall we add a new method def title: String
in the trait and included it here like == Extended Information ($title) ==
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great idea - done.
BTW,
|
@dongjoon-hyun thank you for pointing the ci failure out. I've addressed it by excluding the newly added trait from the compatibility check (it is not needed by the client). |
thanks, merging to master! |
Thank you @cloud-fan @dongjoon-hyun for the review! |
Thank you, @parthchandra and @cloud-fan . |
What changes were proposed in this pull request?
This addresses SPARK-47289 and adds a new section in explain plan where Spark extensions can add additional information for end users. The section is included in the output only if the relevant configuration is enabled and if the extension actually adds some new information
Why are the changes needed?
Extensions to Spark can add their own planning rules and sometimes may need to add additional information about how the plan was generated. This is useful for end users in determining if the extensions rules are working as intended.
Does this PR introduce any user-facing change?
This PR increases the information logged in the UI in the query plan. The attached screenshot shows output from an extension which provides some of its own operations but does not support some operations.
How was this patch tested?
Unit test and manual testing
Was this patch authored or co-authored using generative AI tooling?
No