cloud-fan commented on code in PR #50921:
URL: https://github.com/apache/spark/pull/50921#discussion_r2184500117


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala:
##########
@@ -121,6 +123,103 @@ case class JDBCScanBuilder(
     }
   }
 
+  override def isOtherSideCompatibleForJoin(other: SupportsPushDownJoin): 
Boolean = {
+    other.isInstanceOf[JDBCScanBuilder] &&
+      jdbcOptions.url == other.asInstanceOf[JDBCScanBuilder].jdbcOptions.url
+  };
+
+  // When getJoinedSchema is called, schema shouldn't be pruned yet because 
pushDownJoin API call
+  // can fail. For this reason, we are temporarily holding pruned schema in 
new variable that is
+  // later used in pushDownJoin when crafting the SQL query.
+  var aboutToBePrunedSchema: StructType = finalSchema
+
+  override def getJoinedSchema(
+    other: SupportsPushDownJoin,
+    requiredSchema: StructType,
+    otherSideRequiredSchema: StructType): StructType = {
+    aboutToBePrunedSchema = requiredSchema
+    other.asInstanceOf[JDBCScanBuilder].aboutToBePrunedSchema = 
otherSideRequiredSchema
+
+    var joinedSchema = StructType(Seq())
+
+    (requiredSchema.fields ++ otherSideRequiredSchema.fields)
+      .zipWithIndex
+      .foreach { case (field, idx) =>
+        val newFieldName = JoinOutputAliasIterator.generateColumnAlias
+        joinedSchema =
+          joinedSchema.add(newFieldName, field.dataType, field.nullable, 
field.metadata)
+      }
+
+    joinedSchema
+  }
+
+  override def pushDownJoin(
+    other: SupportsPushDownJoin,
+    requiredSchema: StructType,
+    joinType: JoinType,
+    condition: Predicate): Boolean = {
+    if (!jdbcOptions.pushDownJoin || !dialect.supportsJoin) return false
+    val otherJdbcScanBuilder = other.asInstanceOf[JDBCScanBuilder]
+
+    val requiredOutput = 
requiredSchema.fields.take(aboutToBePrunedSchema.length).map(_.name)
+    val otherSideRequiredOutput =
+      requiredSchema.fields.drop(aboutToBePrunedSchema.length).map(_.name)
+
+    val sqlQuery = buildSQLQueryUsedInJoinPushDown(requiredOutput)
+    val otherSideSqlQuery = otherJdbcScanBuilder
+      .buildSQLQueryUsedInJoinPushDown(otherSideRequiredOutput)
+
+    val joinOutputColumnsString =
+      requiredSchema.fields.map(f => 
dialect.quoteIdentifier(f.name)).mkString(",")
+
+    val joinTypeString = joinType match {
+      case JoinType.INNER_JOIN => "INNER JOIN"
+      case _ => ""
+    }
+
+    if (joinTypeString.isEmpty) return false
+
+    val compiledCondition = dialect.compileExpression(condition)
+    if (!compiledCondition.isDefined) return false
+
+    val conditionString = compiledCondition.get
+
+    val joinQuery = s"""
+       |SELECT $joinOutputColumnsString FROM
+       |($sqlQuery) ${JoinOutputAliasIterator.generateSubqueryQualifier}

Review Comment:
   does the subquery alias matter here given the column names won't have 
conflict? Can we also use `t1` and `t2`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to