You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/06/17 22:41:14 UTC

[GitHub] [spark] sunchao commented on a diff in pull request #35965: [SPARK-38647][SQL] Add SupportsReportOrdering mix in interface for Scan (DataSourceV2)

sunchao commented on code in PR #35965:
URL: https://github.com/apache/spark/pull/35965#discussion_r900534049


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioningAndOrdering.scala:
##########
@@ -21,18 +21,28 @@ import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.connector.catalog.FunctionCatalog
-import org.apache.spark.sql.connector.read.SupportsReportPartitioning
+import org.apache.spark.sql.connector.read.{SupportsReportOrdering, SupportsReportPartitioning}
 import org.apache.spark.sql.connector.read.partitioning.{KeyGroupedPartitioning, UnknownPartitioning}
 import org.apache.spark.util.collection.Utils.sequenceToOption
 
 /**
  * Extracts [[DataSourceV2ScanRelation]] from the input logical plan, converts any V2 partitioning
- * reported by data sources to their catalyst counterparts. Then, annotates the plan with the
- * result.
+ * and ordering reported by data sources to their catalyst counterparts. Then, annotates the plan
+ * with the partitioning and ordering result.
  */
-object V2ScanPartitioning extends Rule[LogicalPlan] with SQLConfHelper {
-  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
-    case d @ DataSourceV2ScanRelation(relation, scan: SupportsReportPartitioning, _, None) =>
+object V2ScanPartitioningAndOrdering extends Rule[LogicalPlan] with SQLConfHelper {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    val scanRules = Seq[LogicalPlan => LogicalPlan] (

Review Comment:
   nit nit: can we keep this one-line?
   ```scala
   val scanRules = Seq[LogicalPlan => LogicalPlan](partitioning, ordering)
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org