alamb commented on code in PR #9780:
URL: https://github.com/apache/arrow-datafusion/pull/9780#discussion_r1546316429


##########
datafusion/common/src/tree_node.rs:
##########
@@ -353,13 +425,38 @@ pub trait TreeNode: Sized {
     }
 
     /// Apply the closure `F` to the node's children.
+    ///
+    /// See `mutate_children` for rewriting in place
     fn apply_children<F: FnMut(&Self) -> Result<TreeNodeRecursion>>(
         &self,
         f: &mut F,
     ) -> Result<TreeNodeRecursion>;
 
-    /// Apply transform `F` to the node's children. Note that the transform `F`
-    /// might have a direction (pre-order or post-order).
+    /// Rewrite the node's children in place using `F`.

Review Comment:
   I chose to leave `mutate_children` unimplemented for `Expr` and the other 
`TreeNodes` initially to reduce the size of this PR (and thus the review 
burden). I will file a follow on ticket to track doing so



##########
datafusion/optimizer/src/optimizer.rs:
##########
@@ -274,20 +276,98 @@ impl Optimizer {
     pub fn with_rules(rules: Vec<Arc<dyn OptimizerRule + Send + Sync>>) -> 
Self {
         Self { rules }
     }
+}
+
+struct Mutator<'a> {
+    apply_order: ApplyOrder,
+    rule: &'a dyn OptimizerRule,
+    config: &'a dyn OptimizerConfig,
+}
+
+impl<'a> TreeNodeMutator for Mutator<'a> {
+    type Node = LogicalPlan;
 
+    fn f_down(&mut self, node: &mut Self::Node) -> Result<Transformed<()>> {
+        if self.apply_order == ApplyOrder::TopDown {
+            optimize_in_place(node, self.rule, self.config)
+        } else {
+            Ok(Transformed::no(()))
+        }
+    }
+
+    fn f_up(&mut self, node: &mut Self::Node) -> Result<Transformed<()>> {
+        if self.apply_order == ApplyOrder::BottomUp {
+            optimize_in_place(node, self.rule, self.config)
+        } else {
+            Ok(Transformed::no(()))
+        }
+    }
+}
+
+/// Applies rule to `plan` in place, returning Transformed with the rewritten
+/// plan
+fn rewrite_in_place(
+    mut plan: LogicalPlan,
+    rule: &dyn OptimizerRule,
+    config: &dyn OptimizerConfig,
+) -> Result<Transformed<LogicalPlan>> {
+    let transformed = match rule.apply_order() {
+        Some(apply_order) => {
+            // use &mut to rewrite plan in place
+            plan.mutate(&mut Mutator {
+                apply_order,
+                rule,
+                config,
+            })
+        }
+        None => optimize_in_place(&mut plan, rule, config),
+    }
+    // convert to bool to drop mut borrow on plan
+    .map(|tnr| tnr.transformed);
+
+    // take back ownership
+    transformed.map(|transformed| {
+        if transformed {
+            Transformed::yes(plan)
+        } else {
+            Transformed::no(plan)
+        }
+    })
+}
+
+/// Invokes the Optimizer rule to rewrite the LogicalPlan in place.
+fn optimize_in_place(
+    plan: &mut LogicalPlan,
+    rule: &dyn OptimizerRule,
+    config: &dyn OptimizerConfig,
+) -> Result<Transformed<()>> {
+    // TODO: introduce a better API to OptimizerRule to allow rewriting in 
place

Review Comment:
   I will file a follow on ticket to support reducing even more of this copying



##########
datafusion/common/src/tree_node.rs:
##########
@@ -174,6 +182,70 @@ pub trait TreeNode: Sized {
         })
     }
 
+    /// Implements the [visitor 
pattern](https://en.wikipedia.org/wiki/Visitor_pattern) for

Review Comment:
   Here is the new TreeNode API to rewrite the nodes in place
   
   I also tried  `Transformed<&mut Node>`, but this got caught up in the borrow 
checker. Returning `Transformed<()>` seemed to work and made using this API 
feasible
   
   @peter-toth  and @berkaysynnada  I would love any feedback on this API that 
you may have



##########
datafusion/optimizer/src/optimizer.rs:
##########
@@ -299,44 +379,65 @@ impl Optimizer {
             log_plan(&format!("Optimizer input (pass {i})"), &new_plan);
 
             for rule in &self.rules {
-                let result =
-                    self.optimize_recursively(rule, &new_plan, config)
-                        .and_then(|plan| {
-                            if let Some(plan) = &plan {
-                                assert_schema_is_the_same(rule.name(), plan, 
&new_plan)?;
-                            }
-                            Ok(plan)
-                        });
-                match result {
-                    Ok(Some(plan)) => {
+                // If we need to skip failed rules, must copy plan before 
attempting to rewrite
+                // as rewriting is destructive
+                let prev_plan = options

Review Comment:
   Now we only copy the (entire!) plan if we need to be able to rewrite it. 
Otherwise no copy is done



##########
datafusion/common/src/tree_node.rs:
##########
@@ -532,6 +669,25 @@ impl<T> Transformed<T> {
     }
 }
 
+impl Transformed<()> {
+    /// Invoke the given function `f`  and combine the transformed state with
+    /// the current state:
+    ///
+    /// * if `f` returns an Err, returns that err
+    ///
+    /// * If `f` returns Ok, sets `self.transformed` to `true` if either self 
or
+    /// the result of `f` were transformed.
+    pub fn and_then<F>(self, f: F) -> Result<Transformed<()>>

Review Comment:
   Introducing this function allowed the implementation of mutate and mutate 
children to be more concise. I couldn't figure out how to implement it in 
general for two `Transformed` as there are two different payloads



##########
datafusion/expr/src/logical_plan/mutate.rs:
##########
@@ -0,0 +1,346 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use super::plan::*;
+use crate::expr::{Exists, InSubquery};
+use crate::{Expr, UserDefinedLogicalNode};
+use datafusion_common::tree_node::Transformed;
+use datafusion_common::{internal_err, Result};
+use datafusion_common::{Column, DFSchema, DFSchemaRef};
+use std::sync::{Arc, OnceLock};
+
+impl LogicalPlan {
+    /// applies `f` to each expression of this node, potentially rewriting it 
in
+    /// place
+    ///
+    /// If `f` returns an error, the error is returned and the expressions are
+    /// left in a partially modified state
+    pub fn rewrite_exprs<F>(&mut self, mut f: F) -> Result<Transformed<()>>
+    where
+        F: FnMut(&mut Expr) -> Result<Transformed<()>>,
+    {
+        match self {
+            LogicalPlan::Projection(Projection { expr, .. }) => {
+                rewrite_expr_iter_mut(expr.iter_mut(), f)
+            }
+            LogicalPlan::Values(Values { values, .. }) => {
+                rewrite_expr_iter_mut(values.iter_mut().flatten(), f)
+            }
+            LogicalPlan::Filter(Filter { predicate, .. }) => f(predicate),
+            LogicalPlan::Repartition(Repartition {
+                partitioning_scheme,
+                ..
+            }) => match partitioning_scheme {
+                Partitioning::Hash(expr, _) => 
rewrite_expr_iter_mut(expr.iter_mut(), f),
+                Partitioning::DistributeBy(expr) => {
+                    rewrite_expr_iter_mut(expr.iter_mut(), f)
+                }
+                Partitioning::RoundRobinBatch(_) => Ok(Transformed::no(())),
+            },
+            LogicalPlan::Window(Window { window_expr, .. }) => {
+                rewrite_expr_iter_mut(window_expr.iter_mut(), f)
+            }
+            LogicalPlan::Aggregate(Aggregate {
+                group_expr,
+                aggr_expr,
+                ..
+            }) => {
+                let exprs = group_expr.iter_mut().chain(aggr_expr.iter_mut());
+                rewrite_expr_iter_mut(exprs, f)
+            }
+            // There are two part of expression for join, equijoin(on) and 
non-equijoin(filter).
+            // 1. the first part is `on.len()` equijoin expressions, and the 
struct of each expr is `left-on = right-on`.
+            // 2. the second part is non-equijoin(filter).
+            LogicalPlan::Join(Join { on, filter, .. }) => {
+                let exprs = on
+                    .iter_mut()
+                    .flat_map(|(e1, e2)| 
std::iter::once(e1).chain(std::iter::once(e2)));
+
+                let result = rewrite_expr_iter_mut(exprs, &mut f)?;
+
+                if let Some(filter) = filter.as_mut() {
+                    result.and_then(|| f(filter))
+                } else {
+                    Ok(result)
+                }
+            }
+            LogicalPlan::Sort(Sort { expr, .. }) => {
+                rewrite_expr_iter_mut(expr.iter_mut(), f)
+            }
+            LogicalPlan::Extension(extension) => {
+                rewrite_extension_exprs(&mut extension.node, f)
+            }
+            LogicalPlan::TableScan(TableScan { filters, .. }) => {
+                rewrite_expr_iter_mut(filters.iter_mut(), f)
+            }
+            LogicalPlan::Unnest(Unnest { column, .. }) => 
rewrite_column(column, f),
+            LogicalPlan::Distinct(Distinct::On(DistinctOn {
+                on_expr,
+                select_expr,
+                sort_expr,
+                ..
+            })) => {
+                let exprs = on_expr
+                    .iter_mut()
+                    .chain(select_expr.iter_mut())
+                    .chain(sort_expr.iter_mut().flat_map(|x| x.iter_mut()));
+
+                rewrite_expr_iter_mut(exprs, f)
+            }
+            // plans without expressions
+            LogicalPlan::EmptyRelation(_)
+            | LogicalPlan::RecursiveQuery(_)
+            | LogicalPlan::Subquery(_)
+            | LogicalPlan::SubqueryAlias(_)
+            | LogicalPlan::Limit(_)
+            | LogicalPlan::Statement(_)
+            | LogicalPlan::CrossJoin(_)
+            | LogicalPlan::Analyze(_)
+            | LogicalPlan::Explain(_)
+            | LogicalPlan::Union(_)
+            | LogicalPlan::Distinct(Distinct::All(_))
+            | LogicalPlan::Dml(_)
+            | LogicalPlan::Ddl(_)
+            | LogicalPlan::Copy(_)
+            | LogicalPlan::DescribeTable(_)
+            | LogicalPlan::Prepare(_) => Ok(Transformed::no(())),
+        }
+    }
+
+    /// applies `f` to each input of this node, rewriting them in place.
+    ///
+    /// # Notes
+    /// Inputs include both direct children as well as any embedded subquery
+    /// `LogicalPlan`s, for example such as are in [`Expr::Exists`].
+    ///
+    /// If `f` returns an `Err`, that Err is returned, and the inputs are left
+    /// in a partially modified state
+    pub fn rewrite_inputs<F>(&mut self, mut f: F) -> Result<Transformed<()>>
+    where
+        F: FnMut(&mut LogicalPlan) -> Result<Transformed<()>>,
+    {
+        let children_result = match self {
+            LogicalPlan::Projection(Projection { input, .. }) => {
+                rewrite_arc(input, &mut f)
+            }
+            LogicalPlan::Filter(Filter { input, .. }) => rewrite_arc(input, 
&mut f),
+            LogicalPlan::Repartition(Repartition { input, .. }) => {
+                rewrite_arc(input, &mut f)
+            }
+            LogicalPlan::Window(Window { input, .. }) => rewrite_arc(input, 
&mut f),
+            LogicalPlan::Aggregate(Aggregate { input, .. }) => 
rewrite_arc(input, &mut f),
+            LogicalPlan::Sort(Sort { input, .. }) => rewrite_arc(input, &mut 
f),
+            LogicalPlan::Join(Join { left, right, .. }) => {
+                rewrite_arc(left, &mut f)?.and_then(|| rewrite_arc(right, &mut 
f))
+            }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                rewrite_arc(left, &mut f)?.and_then(|| rewrite_arc(right, &mut 
f))
+            }
+            LogicalPlan::Limit(Limit { input, .. }) => rewrite_arc(input, &mut 
f),
+            LogicalPlan::Subquery(Subquery { subquery, .. }) => {
+                rewrite_arc(subquery, &mut f)
+            }
+            LogicalPlan::SubqueryAlias(SubqueryAlias { input, .. }) => {
+                rewrite_arc(input, &mut f)
+            }
+            LogicalPlan::Extension(extension) => {
+                rewrite_extension_inputs(&mut extension.node, &mut f)
+            }
+            LogicalPlan::Union(Union { inputs, .. }) => inputs
+                .iter_mut()
+                .try_fold(Transformed::no(()), |acc, input| {
+                    acc.and_then(|| rewrite_arc(input, &mut f))
+                }),
+            LogicalPlan::Distinct(
+                Distinct::All(input) | Distinct::On(DistinctOn { input, .. }),
+            ) => rewrite_arc(input, &mut f),
+            LogicalPlan::Explain(explain) => rewrite_arc(&mut explain.plan, 
&mut f),
+            LogicalPlan::Analyze(analyze) => rewrite_arc(&mut analyze.input, 
&mut f),
+            LogicalPlan::Dml(write) => rewrite_arc(&mut write.input, &mut f),
+            LogicalPlan::Copy(copy) => rewrite_arc(&mut copy.input, &mut f),
+            LogicalPlan::Ddl(ddl) => {
+                if let Some(input) = ddl.input_mut() {
+                    rewrite_arc(input, &mut f)
+                } else {
+                    Ok(Transformed::no(()))
+                }
+            }
+            LogicalPlan::Unnest(Unnest { input, .. }) => rewrite_arc(input, 
&mut f),
+            LogicalPlan::Prepare(Prepare { input, .. }) => rewrite_arc(input, 
&mut f),
+            LogicalPlan::RecursiveQuery(RecursiveQuery {
+                static_term,
+                recursive_term,
+                ..
+            }) => rewrite_arc(static_term, &mut f)?
+                .and_then(|| rewrite_arc(recursive_term, &mut f)),
+            // plans without inputs
+            LogicalPlan::TableScan { .. }
+            | LogicalPlan::Statement { .. }
+            | LogicalPlan::EmptyRelation { .. }
+            | LogicalPlan::Values { .. }
+            | LogicalPlan::DescribeTable(_) => Ok(Transformed::no(())),
+        }?;
+
+        // after visiting the actual children we we need to visit any 
subqueries
+        // that are inside the expressions
+        children_result.and_then(|| self.rewrite_subqueries(&mut f))
+    }
+
+    /// applies `f` to LogicalPlans in any subquery expressions
+    ///
+    /// If Err is returned, the plan may be left in a partially modified state
+    fn rewrite_subqueries<F>(&mut self, mut f: F) -> Result<Transformed<()>>
+    where
+        F: FnMut(&mut LogicalPlan) -> Result<Transformed<()>>,
+    {
+        self.rewrite_exprs(|expr| match expr {
+            Expr::Exists(Exists { subquery, .. })
+            | Expr::InSubquery(InSubquery { subquery, .. })
+            | Expr::ScalarSubquery(subquery) => {
+                rewrite_arc(&mut subquery.subquery, &mut f)
+            }
+            _ => Ok(Transformed::no(())),
+        })
+    }
+}
+
+/// writes each `&mut Expr` in the iterator using `f`
+fn rewrite_expr_iter_mut<'a, F>(
+    i: impl IntoIterator<Item = &'a mut Expr>,
+    mut f: F,
+) -> Result<Transformed<()>>
+where
+    F: FnMut(&mut Expr) -> Result<Transformed<()>>,
+{
+    i.into_iter()
+        .try_fold(Transformed::no(()), |acc, expr| acc.and_then(|| f(expr)))
+}
+
+/// A temporary node that is left in place while rewriting the children of a
+/// [`LogicalPlan`]. This is necessary to ensure that the `LogicalPlan` is
+/// always in a valid state (from the Rust perspective)
+static PLACEHOLDER: OnceLock<Arc<LogicalPlan>> = OnceLock::new();
+
+/// Applies `f` to rewrite the existing node, while avoiding `clone`'ing as 
much

Review Comment:
   This is somewhat 🤮  but required to avoid copies while rewriting `Arc`s



##########
datafusion/optimizer/src/optimizer.rs:
##########
@@ -356,97 +457,22 @@ impl Optimizer {
         debug!("Optimizer took {} ms", start_time.elapsed().as_millis());
         Ok(new_plan)
     }
-
-    fn optimize_node(
-        &self,
-        rule: &Arc<dyn OptimizerRule + Send + Sync>,
-        plan: &LogicalPlan,
-        config: &dyn OptimizerConfig,
-    ) -> Result<Option<LogicalPlan>> {
-        // TODO: future feature: We can do Batch optimize
-        rule.try_optimize(plan, config)
-    }
-
-    fn optimize_inputs(
-        &self,
-        rule: &Arc<dyn OptimizerRule + Send + Sync>,
-        plan: &LogicalPlan,
-        config: &dyn OptimizerConfig,
-    ) -> Result<Option<LogicalPlan>> {
-        let inputs = plan.inputs();
-        let result = inputs
-            .iter()
-            .map(|sub_plan| self.optimize_recursively(rule, sub_plan, config))
-            .collect::<Result<Vec<_>>>()?;
-        if result.is_empty() || result.iter().all(|o| o.is_none()) {
-            return Ok(None);
-        }
-
-        let new_inputs = result
-            .into_iter()
-            .zip(inputs)
-            .map(|(new_plan, old_plan)| match new_plan {
-                Some(plan) => plan,
-                None => old_plan.clone(),
-            })
-            .collect();
-
-        let exprs = plan.expressions();
-        plan.with_new_exprs(exprs, new_inputs).map(Some)
-    }
-
-    /// Use a rule to optimize the whole plan.
-    /// If the rule with `ApplyOrder`, we don't need to recursively handle 
children in rule.
-    pub fn optimize_recursively(
-        &self,
-        rule: &Arc<dyn OptimizerRule + Send + Sync>,
-        plan: &LogicalPlan,
-        config: &dyn OptimizerConfig,
-    ) -> Result<Option<LogicalPlan>> {
-        match rule.apply_order() {
-            Some(order) => match order {
-                ApplyOrder::TopDown => {
-                    let optimize_self_opt = self.optimize_node(rule, plan, 
config)?;
-                    let optimize_inputs_opt = match &optimize_self_opt {
-                        Some(optimized_plan) => {
-                            self.optimize_inputs(rule, optimized_plan, config)?

Review Comment:
   each of these calls *also* copies the plan



##########
datafusion/optimizer/src/analyzer/type_coercion.rs:
##########
@@ -1297,7 +1297,8 @@ mod test {
         let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], 
empty)?);
         dbg!(&plan);
         let expected =
-            "Projection: CAST(Utf8(\"1998-03-18\") AS Timestamp(Nanosecond, 
None)) = CAST(CAST(Utf8(\"1998-03-18\") AS Date32) AS Timestamp(Nanosecond, 
None))\n  EmptyRelation";
+            "Projection: CAST(Utf8(\"1998-03-18\") AS Timestamp(Nanosecond, 
None)) = CAST(CAST(Utf8(\"1998-03-18\") AS Date32) AS Timestamp(Nanosecond, 
None))\

Review Comment:
   I just added some formatting to this test, it is not changed semantically



##########
datafusion/optimizer/src/optimizer.rs:
##########
@@ -274,20 +276,98 @@ impl Optimizer {
     pub fn with_rules(rules: Vec<Arc<dyn OptimizerRule + Send + Sync>>) -> 
Self {
         Self { rules }
     }
+}
+
+struct Mutator<'a> {

Review Comment:
   Here is an example of using the `Mutator` API to rewrite `LogicalPlan`s in 
place 



##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -1062,29 +1063,17 @@ mod tests {
     }
 
     fn assert_optimized_plan_eq_with_rewrite_predicate(
-        plan: &LogicalPlan,
+        plan: LogicalPlan,
         expected: &str,
     ) -> Result<()> {
         let optimizer = Optimizer::with_rules(vec![
             Arc::new(RewriteDisjunctivePredicate::new()),
             Arc::new(PushDownFilter::new()),
         ]);
-        let mut optimized_plan = optimizer
-            .optimize_recursively(
-                optimizer.rules.first().unwrap(),
-                plan,
-                &OptimizerContext::new(),
-            )?
-            .unwrap_or_else(|| plan.clone());
-        optimized_plan = optimizer
-            .optimize_recursively(
-                optimizer.rules.get(1).unwrap(),
-                &optimized_plan,
-                &OptimizerContext::new(),
-            )?
-            .unwrap_or_else(|| plan.clone());
+        let optimized_plan =

Review Comment:
   This now uses the Optimizer directly rather than calling 
`optimize_recursively` (which the Optimizer does internally)



##########
datafusion/optimizer/src/optimizer.rs:
##########
@@ -356,97 +457,22 @@ impl Optimizer {
         debug!("Optimizer took {} ms", start_time.elapsed().as_millis());
         Ok(new_plan)
     }
-
-    fn optimize_node(
-        &self,
-        rule: &Arc<dyn OptimizerRule + Send + Sync>,
-        plan: &LogicalPlan,
-        config: &dyn OptimizerConfig,
-    ) -> Result<Option<LogicalPlan>> {
-        // TODO: future feature: We can do Batch optimize
-        rule.try_optimize(plan, config)
-    }
-
-    fn optimize_inputs(
-        &self,
-        rule: &Arc<dyn OptimizerRule + Send + Sync>,
-        plan: &LogicalPlan,
-        config: &dyn OptimizerConfig,
-    ) -> Result<Option<LogicalPlan>> {
-        let inputs = plan.inputs();
-        let result = inputs
-            .iter()
-            .map(|sub_plan| self.optimize_recursively(rule, sub_plan, config))
-            .collect::<Result<Vec<_>>>()?;
-        if result.is_empty() || result.iter().all(|o| o.is_none()) {
-            return Ok(None);
-        }
-
-        let new_inputs = result
-            .into_iter()
-            .zip(inputs)
-            .map(|(new_plan, old_plan)| match new_plan {
-                Some(plan) => plan,
-                None => old_plan.clone(),
-            })
-            .collect();
-
-        let exprs = plan.expressions();
-        plan.with_new_exprs(exprs, new_inputs).map(Some)

Review Comment:
   this used to copy all expressions and the entire `LogicalPlan` at least once



##########
datafusion/optimizer/src/optimizer.rs:
##########
@@ -506,16 +532,16 @@ mod tests {
             produce_one_row: false,
             schema: Arc::new(DFSchema::empty()),
         });
-        let err = opt.optimize(&plan, &config, &observe).unwrap_err();
+        let err = opt.optimize(plan, &config, &observe).unwrap_err();
         assert_eq!(
             "Optimizer rule 'get table_scan rule' failed\ncaused by\nget 
table_scan rule\ncaused by\n\
             Internal error: Failed due to a difference in schemas, \
-            original schema: DFSchema { fields: [\

Review Comment:
   this test actually shows the output was reversed -- the original schema was 
`[]` not the table schema (which is what is changed by the optimizer)



##########
datafusion/optimizer/src/test/mod.rs:
##########
@@ -152,20 +152,16 @@ pub fn assert_analyzer_check_err(
 }
 pub fn assert_optimized_plan_eq(
     rule: Arc<dyn OptimizerRule + Send + Sync>,
-    plan: &LogicalPlan,
+    plan: LogicalPlan,
     expected: &str,
 ) -> Result<()> {
-    let optimizer = Optimizer::with_rules(vec![rule.clone()]);
-    let optimized_plan = optimizer
-        .optimize_recursively(
-            optimizer.rules.first().unwrap(),
-            plan,
-            &OptimizerContext::new(),
-        )?
-        .unwrap_or_else(|| plan.clone());
+    fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {}
+
+    // in tests we are applying only one rule once
+    let opt_context = OptimizerContext::new().with_max_passes(1);

Review Comment:
   it turns out if you apply this rule multiple times, it changes the output. I 
explicitly made the test only run it a single time to match the previous 
behavior



-- 
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]

Reply via email to