Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -1235,27 +1235,63 @@ class SparkConnectPlanner(val session: SparkSession) {
}
}

private def splitConjunctivePredicates(condition: Expression): Seq[Expression] = {
condition match {
case UnresolvedFunction(Seq("and"), Seq(cond1, cond2), _, _, _) =>
splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2)
case other => other :: Nil
}
}

private def transformJoin(rel: proto.Join): LogicalPlan = {
assert(rel.hasLeft && rel.hasRight, "Both join sides must be present")
if (rel.hasJoinCondition && rel.getUsingColumnsCount > 0) {
throw InvalidPlanInput(
s"Using columns or join conditions cannot be set at the same time in Join")
}
val joinCondition =
if (rel.hasJoinCondition) Some(transformExpression(rel.getJoinCondition)) else None

val catalystJointype = transformJoinType(
if (rel.getJoinType != null) rel.getJoinType else proto.Join.JoinType.JOIN_TYPE_INNER)
val joinType = if (rel.getUsingColumnsCount > 0) {
UsingJoin(catalystJointype, rel.getUsingColumnsList.asScala.toSeq)
} else {
catalystJointype
}
logical.Join(
left = transformRelation(rel.getLeft),
right = transformRelation(rel.getRight),
joinType = joinType,
condition = joinCondition,
hint = logical.JoinHint.NONE)

if (rel.hasJoinCondition) {
val leftDF = Dataset.ofRows(session, transformRelation(rel.getLeft))
val rightDF = Dataset.ofRows(session, transformRelation(rel.getRight))
val joinExprs = splitConjunctivePredicates(transformExpression(rel.getJoinCondition))
.map {
case func @ UnresolvedFunction(Seq(f), Seq(l, r), _, _, _)
if Seq("==", "<=>").contains(f) =>
Copy link
Contributor Author

@zhengruifeng zhengruifeng Jan 25, 2023

Choose a reason for hiding this comment

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

currently, both a == a and a <=> a are taken into account

val l2 = l match {
case UnresolvedAttribute(Seq(c)) => leftDF.apply(c).expr
case other => other
}
val r2 = r match {
case UnresolvedAttribute(Seq(c)) => rightDF.apply(c).expr
case other => other
}
func.copy(arguments = Seq(l2, r2))

case other => other
}
.reduce(And)

leftDF
Copy link
Contributor Author

@zhengruifeng zhengruifeng Jan 25, 2023

Choose a reason for hiding this comment

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

we must use DataFrame.join here to make sure the DataFrame ID will not be changed.

// A globally unique id of this Dataset.
private val id = Dataset.curId.getAndIncrement()

.join(right = rightDF, joinExprs = Column(joinExprs), joinType = joinType.sql)
.logicalPlan

} else {

logical.Join(
left = transformRelation(rel.getLeft),
right = transformRelation(rel.getRight),
joinType = joinType,
condition = None,
hint = logical.JoinHint.NONE)
}
}

private def transformJoinType(t: proto.Join.JoinType): JoinType = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -260,21 +260,21 @@ class SparkConnectPlannerSuite extends SparkFunSuite with SparkConnectPlanTest {
.addArguments(unresolvedAttribute)
.build())

val simpleJoin = proto.Relation.newBuilder
.setJoin(
proto.Join.newBuilder
.setLeft(readRel)
.setRight(readRel)
.setJoinType(proto.Join.JoinType.JOIN_TYPE_INNER)
.setJoinCondition(joinCondition)
.build())
.build()

val res = transform(simpleJoin)
assert(res.nodeName == "Join")
assert(res != null)
val e0 = intercept[AnalysisException] {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

after this PR, Join with JoinCondition will be eagerly analyzed.

val simpleJoin = proto.Relation.newBuilder
.setJoin(
proto.Join.newBuilder
.setLeft(readRel)
.setRight(readRel)
.setJoinType(proto.Join.JoinType.JOIN_TYPE_INNER)
.setJoinCondition(joinCondition)
.build())
.build()
transform(simpleJoin)
}
assert(e0.getMessage.contains("TABLE_OR_VIEW_NOT_FOUND"))

val e = intercept[InvalidPlanInput] {
val e1 = intercept[InvalidPlanInput] {
val simpleJoin = proto.Relation.newBuilder
.setJoin(
proto.Join.newBuilder
Expand All @@ -286,7 +286,7 @@ class SparkConnectPlannerSuite extends SparkFunSuite with SparkConnectPlanTest {
transform(simpleJoin)
}
assert(
e.getMessage.contains(
e1.getMessage.contains(
"Using columns or join conditions cannot be set at the same time in Join"))
}

Expand Down
4 changes: 2 additions & 2 deletions python/pyspark/sql/column.py
Original file line number Diff line number Diff line change
Expand Up @@ -315,9 +315,9 @@ def __ne__( # type: ignore[override]
... Row(value = 'bar'),
... Row(value = None)
... ])
>>> df1.join(df2, df1["value"] == df2["value"]).count() # doctest: +SKIP
>>> df1.join(df2, df1["value"] == df2["value"]).count()
0
>>> df1.join(df2, df1["value"].eqNullSafe(df2["value"])).count() # doctest: +SKIP
>>> df1.join(df2, df1["value"].eqNullSafe(df2["value"])).count()
1
>>> df2 = spark.createDataFrame([
... Row(id=1, value=float('NaN')),
Expand Down
32 changes: 32 additions & 0 deletions python/pyspark/sql/tests/connect/test_connect_basic.py
Original file line number Diff line number Diff line change
Expand Up @@ -337,6 +337,38 @@ def test_join_condition_column_list_columns(self):
)
self.assert_eq(joined_plan3.toPandas(), joined_plan4.toPandas())

def test_join_ambiguous_cols(self):
# SPARK-41812: test join with ambiguous columns
data1 = [Row(id=1, value="foo"), Row(id=2, value=None)]
cdf1 = self.connect.createDataFrame(data1)
sdf1 = self.spark.createDataFrame(data1)

data2 = [Row(value="bar"), Row(value=None)]
cdf2 = self.connect.createDataFrame(data2)
sdf2 = self.spark.createDataFrame(data2)

cdf3 = cdf1.join(cdf2, cdf1["value"] == cdf2["value"])
sdf3 = sdf1.join(sdf2, sdf1["value"] == sdf2["value"])

self.assertEqual(cdf3.schema, sdf3.schema)
self.assertEqual(cdf3.collect(), sdf3.collect())

cdf4 = cdf1.join(cdf2, cdf1["value"].eqNullSafe(cdf2["value"]))
sdf4 = sdf1.join(sdf2, sdf1["value"].eqNullSafe(sdf2["value"]))

self.assertEqual(cdf4.schema, sdf4.schema)
self.assertEqual(cdf4.collect(), sdf4.collect())

cdf5 = cdf1.join(
cdf2, (cdf1["value"] == cdf2["value"]) & (cdf1["value"].eqNullSafe(cdf2["value"]))
)
sdf5 = sdf1.join(
sdf2, (sdf1["value"] == sdf2["value"]) & (sdf1["value"].eqNullSafe(sdf2["value"]))
)

self.assertEqual(cdf5.schema, sdf5.schema)
self.assertEqual(cdf5.collect(), sdf5.collect())

def test_collect(self):
cdf = self.connect.read.table(self.tbl_name)
sdf = self.spark.read.table(self.tbl_name)
Expand Down