-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18120 ][SQL] Call QueryExecutionListener callback methods for … #16664
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 1 commit
751ded0
b0392ed
752125a
ecf9f34
a0c7c22
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,9 +29,10 @@ import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable | |
| import org.apache.spark.sql.execution.QueryExecution | ||
| import org.apache.spark.sql.execution.command.DDLUtils | ||
| import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} | ||
| import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions | ||
| import org.apache.spark.sql.sources.BaseRelation | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.sql.util.{OutputParams, QueryExecutionListener} | ||
| import org.apache.spark.sql.util.{OutputParams} | ||
|
|
||
| /** | ||
| * Interface used to write a [[Dataset]] to external storage systems (e.g. file systems, | ||
|
|
@@ -247,7 +248,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| options = extraOptions.toMap) | ||
|
|
||
| val destination = source match { | ||
| case "jdbc" => extraOptions.get("dbtable") | ||
| case "jdbc" => extraOptions.get(JDBCOptions.JDBC_TABLE_NAME) | ||
| case _ => extraOptions.get("path") | ||
|
Member
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 the external data source connectors, it might not have
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. Yes for methods like saveAsTable() there is no path. Do you see a issue here?
Member
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.
It sounds like
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.
Being the person who requested this class instead of an opaque map, I think using an opaque map makes for a really bad user API. The listener now needs to know about "magic keys" that have special meaning, which can vary depending on the destination. So you end up making up some contract that certain keys have some special meanings an all sources need to use them that way, so basically you end up encoding this class in a map. That being said I'm not super happy with the way JDBC works, because there's still some information embedded in the map. I thought about it a little but didn't come up with a good solution; embedding the table name in the JDBC URI sounds hacky and brittle. Best one I got is a separate field in this class (e.g.
Member
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 think we need to make it more general instead of introducing a class for the write path only.
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.
yes, it is. e.g.
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. Actually all the "magic keys" in the options used by
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.
That's good to know, but they only seem to be, at best, indirectly documented. The
I agree that it needs a careful design and the current one doesn't cover all the options. But this PR is of very marginal value without this information being exposed in some way. If you guys feel strongly that it should be a map and that's it, I guess it will be hard to argue. Then we'll have to do that and document all the keys used internally by Spark and make them public, and promise ourselves that we'll never break them. My belief is that a more structured type would help here. Since the current code is obviously not enough, we could have something that's more future-proof, like: Then listeners can easily handle future params by matching and handling the generic params. Anyway, my opinion is that a raw map is not a very good API, regardless of API stability; it's hard to use and easy to break. But I'll defer to you guys if you really don't like my suggestions.
Member
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. Yes those are public APIs. |
||
| } | ||
| val outputParams = OutputParams(source, destination, extraOptions.toMap) | ||
|
|
@@ -467,7 +468,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { | |
| ) | ||
| val qe = df.sparkSession.sessionState.executePlan( | ||
| CreateTable(tableDesc, mode, Some(df.logicalPlan))) | ||
| val outputParams = new OutputParams(source, Some(tableIdent.unquotedString), extraOptions.toMap) | ||
| val outputParams = OutputParams(source, Some(tableIdent.unquotedString), extraOptions.toMap) | ||
| withAction("saveAsTable", qe, outputParams)(qe.toRdd) | ||
| } | ||
|
|
||
|
|
||
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.
For JDBC, the source value might not be
jdbc. For example,jDbC,JDBC,org.apache.spark.sql.jdbc.DefaultSource,org.apache.spark.sql.jdbcThere 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.
Could you please give me some more info. Looking at the DataFrameWriter#jdbc method it sets the source as "jdbc". Are there other places that this source is being set?
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.
For example,