-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-28344][SQL] detect ambiguous self-join and fail the query #25107
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 all commits
140dfd7
4870abc
65b6762
6b18efe
eec92f6
f5087af
62228e8
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -794,6 +794,13 @@ object SQLConf { | |
| .booleanConf | ||
| .createWithDefault(true) | ||
|
|
||
| val FAIL_AMBIGUOUS_SELF_JOIN = | ||
| buildConf("spark.sql.analyzer.failAmbiguousSelfJoin") | ||
| .doc("When true, fail the Dataset query if it contains ambiguous self-join.") | ||
| .internal() | ||
| .booleanConf | ||
| .createWithDefault(true) | ||
|
|
||
| // Whether to retain group by columns or not in GroupedData.agg. | ||
|
||
| val DATAFRAME_RETAIN_GROUP_COLUMNS = buildConf("spark.sql.retainGroupColumns") | ||
| .internal() | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,7 +30,7 @@ import org.apache.spark.TaskContext | |
| import org.apache.spark.annotation.{DeveloperApi, Evolving, Experimental, Stable, Unstable} | ||
| import org.apache.spark.api.java.JavaRDD | ||
| import org.apache.spark.api.java.function._ | ||
| import org.apache.spark.api.python.{PythonEvalType, PythonRDD, SerDeUtil} | ||
| import org.apache.spark.api.python.{PythonRDD, SerDeUtil} | ||
| import org.apache.spark.api.r.RRDD | ||
| import org.apache.spark.broadcast.Broadcast | ||
| import org.apache.spark.rdd.RDD | ||
|
|
@@ -45,13 +45,15 @@ import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils} | |
| import org.apache.spark.sql.catalyst.plans._ | ||
| import org.apache.spark.sql.catalyst.plans.logical._ | ||
| import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} | ||
| import org.apache.spark.sql.catalyst.trees.TreeNodeTag | ||
| import org.apache.spark.sql.execution._ | ||
| import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} | ||
| import org.apache.spark.sql.execution.command._ | ||
| import org.apache.spark.sql.execution.datasources.LogicalRelation | ||
| import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} | ||
| import org.apache.spark.sql.execution.python.EvaluatePython | ||
| import org.apache.spark.sql.execution.stat.StatFunctions | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.streaming.DataStreamWriter | ||
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.sql.util.SchemaUtils | ||
|
|
@@ -61,6 +63,11 @@ import org.apache.spark.unsafe.types.CalendarInterval | |
| import org.apache.spark.util.Utils | ||
|
|
||
| private[sql] object Dataset { | ||
| val curId = new java.util.concurrent.atomic.AtomicLong() | ||
| val DATASET_ID_KEY = "__dataset_id" | ||
| val COL_POS_KEY = "__col_position" | ||
| val DATASET_ID_TAG = TreeNodeTag[Long]("dataset_id") | ||
|
|
||
| def apply[T: Encoder](sparkSession: SparkSession, logicalPlan: LogicalPlan): Dataset[T] = { | ||
| val dataset = new Dataset(sparkSession, logicalPlan, implicitly[Encoder[T]]) | ||
| // Eagerly bind the encoder so we verify that the encoder matches the underlying | ||
|
|
@@ -182,6 +189,9 @@ class Dataset[T] private[sql]( | |
| @DeveloperApi @Unstable @transient val encoder: Encoder[T]) | ||
| extends Serializable { | ||
|
|
||
| // A globally unique id of this Dataset. | ||
| private val id = Dataset.curId.getAndIncrement() | ||
|
|
||
| queryExecution.assertAnalyzed() | ||
|
|
||
| // Note for Spark contributors: if adding or updating any action in `Dataset`, please make sure | ||
|
|
@@ -198,14 +208,18 @@ class Dataset[T] private[sql]( | |
| @transient private[sql] val logicalPlan: LogicalPlan = { | ||
| // For various commands (like DDL) and queries with side effects, we force query execution | ||
| // to happen right away to let these side effects take place eagerly. | ||
| queryExecution.analyzed match { | ||
| val plan = queryExecution.analyzed match { | ||
| case c: Command => | ||
| LocalRelation(c.output, withAction("command", queryExecution)(_.executeCollect())) | ||
| case u @ Union(children) if children.forall(_.isInstanceOf[Command]) => | ||
| LocalRelation(u.output, withAction("command", queryExecution)(_.executeCollect())) | ||
| case _ => | ||
| queryExecution.analyzed | ||
| } | ||
| if (sparkSession.sessionState.conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN)) { | ||
| plan.setTagValue(Dataset.DATASET_ID_TAG, id) | ||
| } | ||
| plan | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -1311,11 +1325,29 @@ class Dataset[T] private[sql]( | |
| if (sqlContext.conf.supportQuotedRegexColumnName) { | ||
| colRegex(colName) | ||
| } else { | ||
| val expr = resolve(colName) | ||
| Column(expr) | ||
| Column(addDataFrameIdToCol(resolve(colName))) | ||
| } | ||
| } | ||
|
|
||
| // Attach the dataset id and column position to the column reference, so that we can detect | ||
| // ambiguous self-join correctly. See the rule `DetectAmbiguousSelfJoin`. | ||
| // This must be called before we return a `Column` that contains `AttributeReference`. | ||
|
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. Also, document these metadata will be removed in the rule DetectAmbbiguousSelfJoin |
||
| // Note that, the metadata added here are only avaiable in the analyzer, as the analyzer rule | ||
| // `DetectAmbiguousSelfJoin` will remove it. | ||
| private def addDataFrameIdToCol(expr: NamedExpression): NamedExpression = { | ||
| val newExpr = expr transform { | ||
| case a: AttributeReference | ||
| if sparkSession.sessionState.conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN) => | ||
| val metadata = new MetadataBuilder() | ||
| .withMetadata(a.metadata) | ||
| .putLong(Dataset.DATASET_ID_KEY, id) | ||
| .putLong(Dataset.COL_POS_KEY, logicalPlan.output.indexWhere(a.semanticEquals)) | ||
| .build() | ||
| a.withMetadata(metadata) | ||
| } | ||
| newExpr.asInstanceOf[NamedExpression] | ||
| } | ||
|
|
||
| /** | ||
| * Selects column based on the column name specified as a regex and returns it as [[Column]]. | ||
| * @group untypedrel | ||
|
|
@@ -1329,7 +1361,7 @@ class Dataset[T] private[sql]( | |
| case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) => | ||
| Column(UnresolvedRegex(columnNameRegex, Some(nameParts), caseSensitive)) | ||
| case _ => | ||
| Column(resolve(colName)) | ||
| Column(addDataFrameIdToCol(resolve(colName))) | ||
| } | ||
| } | ||
|
|
||
|
|
||
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.
Would
FAIL_AMBIGUOUS_DATASET_COLUMN_REFbe more accurate?And in the doc below, we can describe when this usually happens (e.g., with self-join).
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.
A self-join that contains ambiguous column reference is an ambiguous self-join. I think ambiguous self-join is shorter and easier to understand, as a config name.
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.
BTW there are no other places that can make column reference ambiguous. Self-join is the only place.