Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
address comments
fix scala style
  • Loading branch information
zhengruifeng committed Apr 7, 2024
commit eda211cda0b4d052c6e37ff396773029677989cf
Original file line number Diff line number Diff line change
Expand Up @@ -940,9 +940,10 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
}
assert(e3.getMessage.contains("AMBIGUOUS_COLUMN_REFERENCE"))

// df1.join(df1_filter, df1("i") === 1) fails in classic spark due to:
// org.apache.spark.sql.AnalysisException: Column i#24 are ambiguous
// TODO(SPARK-47749): Dataframe.collect should accept duplicated column names
assert(
// df1.join(df1_filter, df1("i") === 1) fails in classic spark due to:
// org.apache.spark.sql.AnalysisException: Column i#24 are ambiguous
df1.join(df1_filter, df1("i") === 1).columns ===
Array("i", "j", "i", "j"))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -527,41 +527,46 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
logDebug(s"Extract plan_id $planId from $u")

val isMetadataAccess = u.getTagValue(LogicalPlan.IS_METADATA_COL).nonEmpty
val resolved = resolveDataFrameColumnByPlanId(u, planId, isMetadataAccess, q, 0).map(_._1)
if (resolved.isEmpty) {
if (!q.exists(_.exists(_.getTagValue(LogicalPlan.PLAN_ID_TAG).contains(planId)))) {
// Can not find the target plan node with plan id, e.g.
// df1 = spark.createDataFrame([Row(a = 1, b = 2, c = 3)]])
// df2 = spark.createDataFrame([Row(a = 1, b = 2)]])
// df1.select(df2.a) <- illegal reference df2.a
throw QueryCompilationErrors.cannotResolveDataFrameColumn(u)
}

val (resolved, matched) = resolveDataFrameColumnByPlanId(
u, planId, isMetadataAccess, q, 0)
if (!matched) {
// Can not find the target plan node with plan id, e.g.
// df1 = spark.createDataFrame([Row(a = 1, b = 2, c = 3)]])
// df2 = spark.createDataFrame([Row(a = 1, b = 2)]])
// df1.select(df2.a) <- illegal reference df2.a
throw QueryCompilationErrors.cannotResolveDataFrameColumn(u)
}
resolved
resolved.map(_._1)
}

private def resolveDataFrameColumnByPlanId(
u: UnresolvedAttribute,
id: Long,
isMetadataAccess: Boolean,
q: Seq[LogicalPlan],
currentDepth: Int): Option[(NamedExpression, Int)] = {
q.flatMap(resolveDataFrameColumnRecursively(u, id, isMetadataAccess, _, currentDepth))
.sortBy(_._2) // make sure 0-depth result is on the left side (avoid depths like: 1, 2, 0)
currentDepth: Int): (Option[(NamedExpression, Int)], Boolean) = {
val resolved = q.map(resolveDataFrameColumnRecursively(
u, id, isMetadataAccess, _, currentDepth))
val merged = resolved
.flatMap(_._1)
.sortBy(_._2) // sort by depth
.foldLeft(Option.empty[(NamedExpression, Int)]) {
case (None, (r2, d2)) => Some((r2, d2))
case (Some((r1, 0)), (r2, d2)) if d2 != 0 => Some((r1, 0))
case _ => throw QueryCompilationErrors.ambiguousColumnReferences(u)
}
val matched = resolved.exists(_._2)
(merged, matched)
}

private def resolveDataFrameColumnRecursively(
u: UnresolvedAttribute,
id: Long,
isMetadataAccess: Boolean,
p: LogicalPlan,
currentDepth: Int): Option[(NamedExpression, Int)] = {
val resolved = if (p.getTagValue(LogicalPlan.PLAN_ID_TAG).contains(id)) {
currentDepth: Int): (Option[(NamedExpression, Int)], Boolean) = {
val (resolved, matched) = if (p.getTagValue(LogicalPlan.PLAN_ID_TAG).contains(id)) {
val resolved = try {
if (!isMetadataAccess) {
p.resolve(u.nameParts, conf.resolver)
Expand All @@ -575,7 +580,7 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
logDebug(s"Fail to resolve $u with $p due to $e")
None
}
resolved.map(r => (r, currentDepth))
(resolved.map(r => (r, currentDepth)), true)
} else {
resolveDataFrameColumnByPlanId(u, id, isMetadataAccess, p.children, currentDepth + 1)
}
Expand Down Expand Up @@ -605,13 +610,14 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase {
// maybe filtered out here. In this case, resolveDataFrameColumnByPlanId
// returns None, the dataframe column will remain unresolved, and the analyzer
// will try to resolve it without plan id later.
resolved.filter { r =>
val filtered = resolved.filter { r =>
if (isMetadataAccess) {
r._1.references.subsetOf(AttributeSet(p.output ++ p.metadataOutput))
} else {
r._1.references.subsetOf(p.outputSet)
}
}
(filtered, matched)
}

private def resolveDataFrameStar(
Expand Down