-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23303][SQL] improve the explain result for data source v2 relations #20647
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
Changes from 7 commits
a5171e6
dbee281
fc29f8f
dfe603d
a73370a
8c5b934
c5af52e
6fe7681
1164eec
1a96d14
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
This file was deleted.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,15 +35,12 @@ case class DataSourceV2Relation( | |
| options: Map[String, String], | ||
| projection: Seq[AttributeReference], | ||
| filters: Option[Seq[Expression]] = None, | ||
| userSpecifiedSchema: Option[StructType] = None) extends LeafNode with MultiInstanceRelation { | ||
| userSpecifiedSchema: Option[StructType] = None) | ||
| extends LeafNode with MultiInstanceRelation with DataSourceV2StringFormat { | ||
|
|
||
| import DataSourceV2Relation._ | ||
|
|
||
| override def simpleString: String = { | ||
| s"DataSourceV2Relation(source=${source.name}, " + | ||
| s"schema=[${output.map(a => s"$a ${a.dataType.simpleString}").mkString(", ")}], " + | ||
| s"filters=[${pushedFilters.mkString(", ")}], options=$options)" | ||
| } | ||
| override def simpleString: String = "RelationV2 " + metadataString | ||
|
|
||
| override lazy val schema: StructType = reader.readSchema() | ||
|
|
||
|
|
@@ -107,19 +104,36 @@ case class DataSourceV2Relation( | |
| } | ||
|
|
||
| /** | ||
| * A specialization of DataSourceV2Relation with the streaming bit set to true. Otherwise identical | ||
| * to the non-streaming relation. | ||
| * A specialization of [[DataSourceV2Relation]] with the streaming bit set to true. | ||
| * | ||
| * Note that, this plan has a mutable reader, so Spark won't apply operator push-down for this plan, | ||
| * to avoid making the plan mutable. We should consolidate this plan and [[DataSourceV2Relation]] | ||
| * after we figure out how to apply operator push-down for streaming data sources. | ||
|
||
| */ | ||
| case class StreamingDataSourceV2Relation( | ||
| output: Seq[AttributeReference], | ||
| source: DataSourceV2, | ||
| options: Map[String, String], | ||
| reader: DataSourceReader) | ||
| extends LeafNode with DataSourceReaderHolder with MultiInstanceRelation { | ||
| extends LeafNode with MultiInstanceRelation with DataSourceV2StringFormat { | ||
|
|
||
| override def isStreaming: Boolean = true | ||
|
|
||
| override def canEqual(other: Any): Boolean = other.isInstanceOf[StreamingDataSourceV2Relation] | ||
| override def simpleString: String = "Streaming RelationV2 " + metadataString | ||
|
|
||
| override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) | ||
|
|
||
| // TODO: unify the equal/hashCode implementation for all data source v2 query plans. | ||
| override def equals(other: Any): Boolean = other match { | ||
| case other: StreamingDataSourceV2Relation => | ||
| output == other.output && reader.getClass == other.reader.getClass && options == other.options | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Now it's exactly same as before. We should clean it up after figure out how to push down operators to streaming relation. |
||
| case _ => false | ||
| } | ||
|
|
||
| override def hashCode(): Int = { | ||
| Seq(output, source, options).hashCode() | ||
| } | ||
|
|
||
| override def computeStats(): Statistics = reader match { | ||
| case r: SupportsReportStatistics => | ||
| Statistics(sizeInBytes = r.getStatistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,11 +23,11 @@ import org.apache.spark.sql.execution.SparkPlan | |
|
|
||
| object DataSourceV2Strategy extends Strategy { | ||
| override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { | ||
| case relation: DataSourceV2Relation => | ||
| DataSourceV2ScanExec(relation.output, relation.reader) :: Nil | ||
| case r: DataSourceV2Relation => | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why rename this variable?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think this change is necessary and it just makes the patch larger.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Strictly speaking, if I don't change this line, then I need to split the next line into 2 lines, as I need to pass more parameters, so it's still a 2-line diff. I think tiny things like this is really not worth to be pointed during code review, we should save our effort to focus more on the actual code logic. What do you think?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm pointing it out because I think it is significant. This is also why I sent a note to the dev list. Changes like this make it much harder to work with Spark because little things change that are not necessary and cause conflicts. That's a problem not just when I'm maintaining a branch, but also when I'm trying to get a PR committed. These changes cause more work because we have to rebase PRs and update for variables that have changed names in the name of style. In this particular case, I would probably wrap the line that is too long. If you want to rename to fix it, I'd consider that reasonable. But all the other cases that aren't necessary should be reverted. |
||
| DataSourceV2ScanExec(r.output, r.source, r.options, r.reader) :: Nil | ||
|
|
||
| case relation: StreamingDataSourceV2Relation => | ||
| DataSourceV2ScanExec(relation.output, relation.reader) :: Nil | ||
| case r: StreamingDataSourceV2Relation => | ||
| DataSourceV2ScanExec(r.output, r.source, r.options, r.reader) :: Nil | ||
|
|
||
| case WriteToDataSourceV2(writer, query) => | ||
| WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,94 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.execution.datasources.v2 | ||
|
|
||
| import org.apache.commons.lang3.StringUtils | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.sources.DataSourceRegister | ||
| import org.apache.spark.sql.sources.v2.DataSourceV2 | ||
| import org.apache.spark.sql.sources.v2.reader._ | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| /** | ||
| * A trait that can be used by data source v2 related query plans(both logical and physical), to | ||
| * provide a string format of the data source information for explain. | ||
| */ | ||
| trait DataSourceV2StringFormat { | ||
|
|
||
| /** | ||
| * The instance of this data source implementation. Note that we only consider its class in | ||
| * equals/hashCode, not the instance itself. | ||
| */ | ||
| def source: DataSourceV2 | ||
|
|
||
| /** | ||
| * The output of the data source reader, w.r.t. column pruning. | ||
| */ | ||
| def output: Seq[Attribute] | ||
|
|
||
| /** | ||
| * The options for this data source reader. | ||
| */ | ||
| def options: Map[String, String] | ||
|
|
||
| /** | ||
| * The created data source reader. Here we use it to get the filters that has been pushed down | ||
| * so far, itself doesn't take part in the equals/hashCode. | ||
| */ | ||
| def reader: DataSourceReader | ||
|
|
||
| private lazy val filters = reader match { | ||
| case s: SupportsPushDownCatalystFilters => s.pushedCatalystFilters().toSet | ||
| case s: SupportsPushDownFilters => s.pushedFilters().toSet | ||
| case _ => Set.empty | ||
| } | ||
|
|
||
| private def sourceName: String = source match { | ||
| case registered: DataSourceRegister => registered.shortName() | ||
| case _ => source.getClass.getSimpleName.stripSuffix("$") | ||
| } | ||
|
|
||
| def metadataString: String = { | ||
| val entries = scala.collection.mutable.ArrayBuffer.empty[(String, String)] | ||
|
|
||
| if (filters.nonEmpty) { | ||
| entries += "Pushed Filters" -> filters.mkString("[", ", ", "]") | ||
|
||
| } | ||
|
|
||
| // TODO: we should only display some standard options like path, table, etc. | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For followup, there are 2 proposals:
|
||
| entries ++= options | ||
|
||
|
|
||
| val outputStr = Utils.truncatedString(output, "[", ", ", "]") | ||
|
|
||
| val entriesStr = if (entries.nonEmpty) { | ||
| Utils.truncatedString(entries.map { | ||
| case (key, value) => StringUtils.abbreviate(redact(key + ":" + value), 100) | ||
|
||
| }, " (", ", ", ")") | ||
| } else { | ||
| "" | ||
| } | ||
|
|
||
| s"$sourceName$outputStr$entriesStr" | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should there be spaces? I don't see spaces added to the components, so it looks like this might look squished together.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| } | ||
|
|
||
| private def redact(text: String): String = { | ||
| Utils.redact(SQLConf.get.stringRedationPattern, text) | ||
| } | ||
| } | ||

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'm -0 on including this in a PR to improve explain results. This could needlessly cause commit conflicts when maintaining a branch. But, this is small and needs to go in somewhere. I would remove it, though.
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.
are you suggesting we should open a JIRA to fix the internal document? We usually fix them in related PRs that touch the 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.
No, I think it is okay, but I would not add it if it were my commit. It is unlikely that this will cause commit conflicts so it should be fine. I would just prefer to have a docs commit later to keep track of this.