You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "mustafasrepo (via GitHub)" <gi...@apache.org> on 2023/04/06 07:54:00 UTC

[GitHub] [arrow-datafusion] mustafasrepo commented on a diff in pull request #5863: Encapsulate `PhysicalSortRequrement` and add more doc comment

mustafasrepo commented on code in PR #5863:
URL: https://github.com/apache/arrow-datafusion/pull/5863#discussion_r1159416812


##########
datafusion/physical-expr/src/sort_expr.rs:
##########
@@ -107,19 +147,87 @@ impl std::fmt::Display for PhysicalSortRequirement {
 }
 
 impl PhysicalSortRequirement {
+    /// Creates a new requirement.
+    ///
+    /// If `options` is `Some(..)`, creates an `exact` requirement,
+    /// which must match both `options` and `expr`.
+    ///
+    /// If `options` is `None`, Creates a new `expr_only` requirement,
+    /// which must match only `expr`.
+    ///
+    /// See [`PhysicalSortRequirement`] for examples.
+    pub fn new(expr: Arc<dyn PhysicalExpr>, options: Option<SortOptions>) -> Self {
+        Self { expr, options }
+    }
+
+    /// Replace the required expression for this requirement with the new one
+    pub fn with_expr(mut self, expr: Arc<dyn PhysicalExpr>) -> Self {
+        self.expr = expr;
+        self
+    }
+
+    /// Converts the `PhysicalSortRequirement` to `PhysicalSortExpr`.
+    /// If required ordering is `None` for an entry, the default
+    /// ordering `ASC, NULLS LAST` is used.
+    ///
+    /// The default is picked to be consistent with
+    /// PostgreSQL: <https://www.postgresql.org/docs/current/queries-order.html>
+    pub fn into_sort_expr(self) -> PhysicalSortExpr {
+        let Self { expr, options } = self;
+
+        let options = options.unwrap_or(SortOptions {
+            descending: false,
+            nulls_first: false,
+        });
+        PhysicalSortExpr { expr, options }
+    }
+
     /// Returns whether this requirement is equal or more specific than `other`.
     pub fn compatible(&self, other: &PhysicalSortRequirement) -> bool {
         self.expr.eq(&other.expr)
             && other.options.map_or(true, |other_opts| {
                 self.options.map_or(false, |opts| opts == other_opts)
             })
     }
-}
 
-pub fn make_sort_requirements_from_exprs(
-    ordering: &[PhysicalSortExpr],
-) -> Vec<PhysicalSortRequirement> {
-    ordering.iter().map(|e| e.clone().into()).collect()
+    /// Returns [`PhysicalSortRequirement`] that requires the exact
+    /// sort of the [`PhysicalSortExpr`]s in `ordering`
+    ///
+    /// This method is designed for
+    /// use implementing [`ExecutionPlan::required_input_ordering`].
+    pub fn from_sort_exprs<'a>(
+        ordering: impl IntoIterator<Item = &'a PhysicalSortExpr>,
+    ) -> Vec<PhysicalSortRequirement> {
+        ordering
+            .into_iter()
+            .map(PhysicalSortRequirement::from)
+            .collect()
+    }
+
+    /// Converts an iterator of [`PhysicalSortRequirement`] into a Vec
+    /// of [`PhysicalSortExpr`]s.
+    ///
+    /// This function converts `PhysicalSortRequirement` to `PhysicalSortExpr`
+    /// for each entry in the input. If required ordering is None for an entry
+    /// default ordering `ASC, NULLS LAST` if given (see [`into_sort_expr`])
+    pub fn to_sort_exprs(
+        requirements: impl IntoIterator<Item = PhysicalSortRequirement>,
+    ) -> Vec<PhysicalSortExpr> {

Review Comment:
   I think we can use     
   ```rust
   pub fn to_sort_exprs<'a>(
           requirements: impl IntoIterator<Item = &'a PhysicalSortRequirement>,
       ) -> Vec<PhysicalSortExpr> 
   ```
   as signature for `to_sort_exprs` as in `from_sort_exprs`. This would help remove a couple clones in the code. 
   



-- 
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: github-unsubscribe@arrow.apache.org

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