Skip to content
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

Closed
wants to merge 8 commits into from

Conversation

parthchandra
Copy link
Contributor

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.

Screenshot 2024-03-11 at 10 23 36 AM

How was this patch tested?

Unit test and manual testing

Was this patch authored or co-authored using generative AI tooling?

No

@github-actions github-actions bot added the SQL label Mar 13, 2024
Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@parthchandra
Copy link
Contributor Author

@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.
Screenshot 2024-03-13 at 10 05 15 AM
I've never encountered this situation in github before. Any suggestions?

@parthchandra
Copy link
Contributor Author

Rebased on latest master and tests are passing now

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a 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] {
Copy link
Contributor

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.

Copy link
Contributor Author

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.

Copy link
Member

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 .

Copy link
Contributor

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.

@parthchandra
Copy link
Contributor Author

CI test failure appears to be unrelated. Not re-running, because this PR will change.

@parthchandra
Copy link
Contributor Author

@cloud-fan I've update this PR. Can you please take a look?
I also moved the unit test to a more appropriate test suite.
(Sorry it took a while; I was making sure that with this approach extensions would be able to publish their extended info and also be able to build and work with older versions of Spark which do not have the trait.)

import scala.io.Source

Copy link
Member

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?

Copy link
Contributor Author

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.
*/

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit. Remove empty line.

Copy link
Contributor Author

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]
Copy link
Member

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?

Copy link
Contributor Author

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")
Copy link
Member

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?

Copy link
Contributor Author

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 {
Copy link
Contributor

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")

Copy link
Contributor Author

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
Copy link
Contributor

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

Copy link
Contributor Author

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)
Copy link
Contributor

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?

Copy link
Contributor Author

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]
Copy link
Contributor

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

Copy link
Contributor Author

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) =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about catch NonFatal?

Copy link
Contributor Author

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")
Copy link
Contributor

@cloud-fan cloud-fan Apr 4, 2024

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) ==?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great idea - done.

@dongjoon-hyun
Copy link
Member

BTW, connect-client-jvm seems to start to fail in CIs.

Do connect-client-jvm module mima check ...
finish connect-client-jvm module mima check ...
ERROR: Comparing Client jar: /__w/spark/spark/connector/connect/client/jvm/target/scala-2.13/spark-connect-client-jvm_2.13-4.0.0-SNAPSHOT.jar and Sql jar: /__w/spark/spark/sql/core/target/scala-2.13/spark-sql_2.13-4.0.0-SNAPSHOT.jar 
problems with Sql module: 
interface org.apache.spark.sql.ExtendedExplainGenerator does not have a correspondent in client version
Exceptions to binary compatibility can be added in 'CheckConnectJvmClientCompatibility#checkMiMaCompatibilityWithSqlModule'
connect-client-jvm module mima check failed.

@parthchandra
Copy link
Contributor Author

@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).

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in d5620cb Apr 5, 2024
@parthchandra
Copy link
Contributor Author

Thank you @cloud-fan @dongjoon-hyun for the review!

@dongjoon-hyun
Copy link
Member

Thank you, @parthchandra and @cloud-fan .
This is much better. :)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants