alamb commented on code in PR #5171:
URL: https://github.com/apache/arrow-datafusion/pull/5171#discussion_r1097703424
##########
datafusion/core/tests/sql/joins.rs:
##########
@@ -1980,8 +1980,8 @@ async fn left_semi_join() -> Result<()> {
let physical_plan = dataframe.create_physical_plan().await?;
let expected = if repartition_joins {
vec![
- "SortExec: [t1_id@0 ASC NULLS LAST]",
- " CoalescePartitionsExec",
+ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]",
+ " SortExec: [t1_id@0 ASC NULLS LAST]",
Review Comment:
is the idea here that SortExec already coalsces within itself so
CoalscePartitionsExec isn't needed?
##########
datafusion/core/tests/sql/window.rs:
##########
@@ -2385,6 +2384,87 @@ async fn
test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re
Ok(())
}
+#[tokio::test]
+async fn test_window_agg_global_sort() -> Result<()> {
+ let config = SessionConfig::new()
+ .with_repartition_windows(false)
+ .with_target_partitions(2);
+ let ctx = SessionContext::with_config(config);
+ register_aggregate_csv(&ctx).await?;
+ let sql = "SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM
aggregate_test_100 ORDER BY c1 ASC";
+
+ let msg = format!("Creating logical plan for '{sql}'");
+ let dataframe = ctx.sql(sql).await.expect(&msg);
+ let physical_plan = dataframe.create_physical_plan().await?;
+ let formatted = displayable(physical_plan.as_ref()).indent().to_string();
+ // Only 1 SortExec was added
+ let expected = {
+ vec![
+ "SortPreservingMergeExec: [c1@0 ASC NULLS LAST]",
+ " ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY
[aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED
FOLLOWING@1 as rn1]",
+ " RepartitionExec: partitioning=RoundRobinBatch(2),
input_partitions=1",
Review Comment:
🤔 I spent some time thinking this through and I think repartitioning a
sorted stream to just remerge it is ok. This plan in particular seems like it
would be better without the repartition / merge because the Projection isn't
doing anything. Perhaps that will be fixed by
https://github.com/apache/arrow-datafusion/pull/5074 from @xiaoyong-z
##########
datafusion/core/src/physical_optimizer/sort_enforcement.rs:
##########
@@ -102,34 +134,170 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort {
.collect::<Result<Vec<_>>>()?;
let children_plans = children_requirements
.iter()
- .map(|elem| elem.plan.clone())
- .collect::<Vec<_>>();
+ .map(|item| item.plan.clone())
+ .collect();
let sort_onwards = children_requirements
- .iter()
- .map(|item| {
- let onwards = &item.sort_onwards;
- if !onwards.is_empty() {
- let flags = item.plan.maintains_input_order();
- // `onwards` starts from sort introducing executor(e.g
`SortExec`, `SortPreservingMergeExec`) till the current executor
- // if the executors in between maintain input
ordering. If we are at
- // the beginning both `SortExec` and
`SortPreservingMergeExec` doesn't maintain ordering(they introduce ordering).
- // However, we want to propagate them above anyway.
- for (maintains, element) in
flags.into_iter().zip(onwards.iter())
- {
- if (maintains || is_sort(&item.plan)) &&
!element.is_empty() {
- return element.clone();
+ .into_iter()
+ .enumerate()
+ .map(|(idx, item)| {
+ let plan = &item.plan;
+ // Leaves of the `sort_onwards` tree are sort-introducing
operators
+ // (e.g `SortExec`, `SortPreservingMergeExec`). This tree
collects
+ // all the intermediate executors that maintain this
ordering. If
+ // we just saw a sort-introducing operator, we reset the
tree and
+ // start accumulating.
+ if is_sort(plan) {
+ return Some(ExecTree {
+ idx,
+ plan: item.plan,
+ children: vec![],
+ });
+ } else if is_limit(plan) {
+ // There is no sort linkage for this path, it starts
at a limit.
+ return None;
+ }
+ let is_spm = is_sort_preserving_merge(plan);
+ let is_union = plan.as_any().is::<UnionExec>();
+ // If the executor is a `UnionExec`, and it has an output
ordering;
+ // then it at least partially maintains some child's
output ordering.
+ // Therefore, we propagate this information upwards.
+ let partially_maintains =
+ is_union && plan.output_ordering().is_some();
Review Comment:
I wonder if there is any way to make these decisions on properties of the
plan (i.e. has multiple inputs and an output ordering) rather than the type of
operator (Union)?
##########
datafusion/core/src/physical_optimizer/sort_enforcement.rs:
##########
@@ -775,6 +1187,133 @@ mod tests {
Ok(())
}
+ #[tokio::test]
+ async fn test_remove_unnecessary_sort2() -> Result<()> {
+ let schema = create_test_schema()?;
+ let source = memory_exec(&schema);
+ let sort_exprs = vec![sort_expr("non_nullable_col", &schema)];
+ let sort = sort_exec(sort_exprs.clone(), source);
+ let spm = sort_preserving_merge_exec(sort_exprs, sort);
+
+ let sort_exprs = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let sort2 = sort_exec(sort_exprs.clone(), spm);
+ let spm2 = sort_preserving_merge_exec(sort_exprs, sort2);
+
+ let sort_exprs = vec![sort_expr("nullable_col", &schema)];
+ let sort3 = sort_exec(sort_exprs, spm2);
+ let physical_plan = repartition_exec(repartition_exec(sort3));
+
+ let expected_input = vec![
+ "RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=10",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=1",
+ " SortExec: [nullable_col@0 ASC]",
+ " SortPreservingMergeExec: [nullable_col@0
ASC,non_nullable_col@1 ASC]",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " SortPreservingMergeExec: [non_nullable_col@1 ASC]",
+ " SortExec: [non_nullable_col@1 ASC]",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+
+ let expected_optimized = vec![
+ "RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=10",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=0",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+ assert_optimized!(expected_input, expected_optimized, physical_plan);
+ Ok(())
+ }
+
+ #[tokio::test]
+ async fn test_remove_unnecessary_sort3() -> Result<()> {
+ let schema = create_test_schema()?;
+ let source = memory_exec(&schema);
+ let sort_exprs = vec![sort_expr("non_nullable_col", &schema)];
+ let sort = sort_exec(sort_exprs.clone(), source);
+ let spm = sort_preserving_merge_exec(sort_exprs, sort);
+
+ let sort_exprs = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let repartition_exec = repartition_exec(spm);
+ let sort2 = sort_exec(sort_exprs.clone(), repartition_exec);
+ let spm2 = sort_preserving_merge_exec(sort_exprs, sort2);
+
+ let physical_plan = aggregate_exec(spm2);
+
+ // When removing a `SortPreservingMergeExec`, make sure that
partitioning
+ // requirements are not violated. In some cases, we may need to replace
+ // it with a `CoalescePartitionsExec` instead of directly removing it.
+ let expected_input = vec![
+ "AggregateExec: mode=Final, gby=[], aggr=[]",
+ " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=1",
+ " SortPreservingMergeExec: [non_nullable_col@1 ASC]",
+ " SortExec: [non_nullable_col@1 ASC]",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+
+ let expected_optimized = vec![
+ "AggregateExec: mode=Final, gby=[], aggr=[]",
+ " CoalescePartitionsExec",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=0",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+ assert_optimized!(expected_input, expected_optimized, physical_plan);
+ Ok(())
+ }
+
+ #[tokio::test]
+ async fn test_do_not_remove_sort_with_limit() -> Result<()> {
+ let schema = create_test_schema()?;
+
+ let source1 = parquet_exec(&schema);
+ let sort_exprs = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let sort = sort_exec(sort_exprs.clone(), source1);
+ let limit = local_limit_exec(sort);
+ let limit = global_limit_exec(limit);
+
+ let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)];
+ let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs);
+
+ let union = union_exec(vec![source2, limit]);
+ let repartition = repartition_exec(union);
+ let physical_plan = sort_preserving_merge_exec(sort_exprs,
repartition);
+
+ let expected_input = vec![
+ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=2",
+ " UnionExec",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
output_ordering=[nullable_col@0 ASC], projection=[nullable_col,
non_nullable_col]",
+ " GlobalLimitExec: skip=0, fetch=100",
+ " LocalLimitExec: fetch=100",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
projection=[nullable_col, non_nullable_col]",
+ ];
+
+ // We should keep the bottom `SortExec`.
+ let expected_optimized = vec![
+ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=2",
+ " UnionExec",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
output_ordering=[nullable_col@0 ASC], projection=[nullable_col,
non_nullable_col]",
+ " GlobalLimitExec: skip=0, fetch=100",
+ " LocalLimitExec: fetch=100",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
+ " ParquetExec: limit=None, partitions={1 group:
[[x]]}, projection=[nullable_col, non_nullable_col]",
+ ];
+ assert_optimized!(expected_input, expected_optimized, physical_plan);
+ Ok(())
+ }
+
#[tokio::test]
Review Comment:
I find this behavior strange -- the fact that sort enforcement and
optimization are done in a single pass. Maybe eventually we could move the "fix
sort order to be correct" as one pass and then "optimize away unecessary sorts"
as a follow on pass
##########
datafusion/core/src/physical_optimizer/sort_enforcement.rs:
##########
@@ -775,6 +1187,133 @@ mod tests {
Ok(())
}
+ #[tokio::test]
+ async fn test_remove_unnecessary_sort2() -> Result<()> {
+ let schema = create_test_schema()?;
+ let source = memory_exec(&schema);
+ let sort_exprs = vec![sort_expr("non_nullable_col", &schema)];
+ let sort = sort_exec(sort_exprs.clone(), source);
+ let spm = sort_preserving_merge_exec(sort_exprs, sort);
+
+ let sort_exprs = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let sort2 = sort_exec(sort_exprs.clone(), spm);
+ let spm2 = sort_preserving_merge_exec(sort_exprs, sort2);
+
+ let sort_exprs = vec![sort_expr("nullable_col", &schema)];
+ let sort3 = sort_exec(sort_exprs, spm2);
+ let physical_plan = repartition_exec(repartition_exec(sort3));
+
+ let expected_input = vec![
+ "RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=10",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=1",
+ " SortExec: [nullable_col@0 ASC]",
+ " SortPreservingMergeExec: [nullable_col@0
ASC,non_nullable_col@1 ASC]",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " SortPreservingMergeExec: [non_nullable_col@1 ASC]",
+ " SortExec: [non_nullable_col@1 ASC]",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+
+ let expected_optimized = vec![
+ "RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=10",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=0",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+ assert_optimized!(expected_input, expected_optimized, physical_plan);
+ Ok(())
+ }
+
+ #[tokio::test]
+ async fn test_remove_unnecessary_sort3() -> Result<()> {
+ let schema = create_test_schema()?;
+ let source = memory_exec(&schema);
+ let sort_exprs = vec![sort_expr("non_nullable_col", &schema)];
+ let sort = sort_exec(sort_exprs.clone(), source);
+ let spm = sort_preserving_merge_exec(sort_exprs, sort);
+
+ let sort_exprs = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let repartition_exec = repartition_exec(spm);
+ let sort2 = sort_exec(sort_exprs.clone(), repartition_exec);
+ let spm2 = sort_preserving_merge_exec(sort_exprs, sort2);
+
+ let physical_plan = aggregate_exec(spm2);
+
+ // When removing a `SortPreservingMergeExec`, make sure that
partitioning
+ // requirements are not violated. In some cases, we may need to replace
+ // it with a `CoalescePartitionsExec` instead of directly removing it.
+ let expected_input = vec![
+ "AggregateExec: mode=Final, gby=[], aggr=[]",
+ " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=1",
+ " SortPreservingMergeExec: [non_nullable_col@1 ASC]",
+ " SortExec: [non_nullable_col@1 ASC]",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+
+ let expected_optimized = vec![
+ "AggregateExec: mode=Final, gby=[], aggr=[]",
+ " CoalescePartitionsExec",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=0",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+ assert_optimized!(expected_input, expected_optimized, physical_plan);
+ Ok(())
+ }
+
+ #[tokio::test]
+ async fn test_do_not_remove_sort_with_limit() -> Result<()> {
+ let schema = create_test_schema()?;
+
+ let source1 = parquet_exec(&schema);
+ let sort_exprs = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let sort = sort_exec(sort_exprs.clone(), source1);
+ let limit = local_limit_exec(sort);
+ let limit = global_limit_exec(limit);
+
+ let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)];
+ let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs);
+
+ let union = union_exec(vec![source2, limit]);
+ let repartition = repartition_exec(union);
+ let physical_plan = sort_preserving_merge_exec(sort_exprs,
repartition);
+
+ let expected_input = vec![
+ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=2",
+ " UnionExec",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
output_ordering=[nullable_col@0 ASC], projection=[nullable_col,
non_nullable_col]",
+ " GlobalLimitExec: skip=0, fetch=100",
+ " LocalLimitExec: fetch=100",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
projection=[nullable_col, non_nullable_col]",
+ ];
+
+ // We should keep the bottom `SortExec`.
+ let expected_optimized = vec![
+ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=2",
+ " UnionExec",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
output_ordering=[nullable_col@0 ASC], projection=[nullable_col,
non_nullable_col]",
+ " GlobalLimitExec: skip=0, fetch=100",
+ " LocalLimitExec: fetch=100",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
+ " ParquetExec: limit=None, partitions={1 group:
[[x]]}, projection=[nullable_col, non_nullable_col]",
+ ];
+ assert_optimized!(expected_input, expected_optimized, physical_plan);
+ Ok(())
+ }
+
#[tokio::test]
async fn test_change_wrong_sorting() -> Result<()> {
Review Comment:
Can I please request you don't change the inputs to the existing tests (aka
keep `test_change_wrong_sorting`, `test_union_inputs_sorted`, ) for this
initial PR review? I would like to know how the code changes have changed the
existing tests.
Once we have merged this PR, if some of the tests are redundant perhaps we
can remove them as a follow on PM
##########
datafusion/core/src/physical_optimizer/sort_enforcement.rs:
##########
@@ -775,6 +1187,133 @@ mod tests {
Ok(())
}
+ #[tokio::test]
+ async fn test_remove_unnecessary_sort2() -> Result<()> {
+ let schema = create_test_schema()?;
+ let source = memory_exec(&schema);
+ let sort_exprs = vec![sort_expr("non_nullable_col", &schema)];
+ let sort = sort_exec(sort_exprs.clone(), source);
+ let spm = sort_preserving_merge_exec(sort_exprs, sort);
+
+ let sort_exprs = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let sort2 = sort_exec(sort_exprs.clone(), spm);
+ let spm2 = sort_preserving_merge_exec(sort_exprs, sort2);
+
+ let sort_exprs = vec![sort_expr("nullable_col", &schema)];
+ let sort3 = sort_exec(sort_exprs, spm2);
+ let physical_plan = repartition_exec(repartition_exec(sort3));
+
+ let expected_input = vec![
+ "RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=10",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=1",
+ " SortExec: [nullable_col@0 ASC]",
+ " SortPreservingMergeExec: [nullable_col@0
ASC,non_nullable_col@1 ASC]",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " SortPreservingMergeExec: [non_nullable_col@1 ASC]",
+ " SortExec: [non_nullable_col@1 ASC]",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+
+ let expected_optimized = vec![
+ "RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=10",
+ " RepartitionExec: partitioning=RoundRobinBatch(10),
input_partitions=0",
+ " MemoryExec: partitions=0, partition_sizes=[]",
+ ];
+ assert_optimized!(expected_input, expected_optimized, physical_plan);
+ Ok(())
+ }
+
+ #[tokio::test]
+ async fn test_remove_unnecessary_sort3() -> Result<()> {
+ let schema = create_test_schema()?;
+ let source = memory_exec(&schema);
+ let sort_exprs = vec![sort_expr("non_nullable_col", &schema)];
+ let sort = sort_exec(sort_exprs.clone(), source);
+ let spm = sort_preserving_merge_exec(sort_exprs, sort);
+
+ let sort_exprs = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let repartition_exec = repartition_exec(spm);
+ let sort2 = sort_exec(sort_exprs.clone(), repartition_exec);
+ let spm2 = sort_preserving_merge_exec(sort_exprs, sort2);
+
+ let physical_plan = aggregate_exec(spm2);
+
+ // When removing a `SortPreservingMergeExec`, make sure that
partitioning
+ // requirements are not violated. In some cases, we may need to replace
+ // it with a `CoalescePartitionsExec` instead of directly removing it.
+ let expected_input = vec![
+ "AggregateExec: mode=Final, gby=[], aggr=[]",
+ " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1
ASC]",
Review Comment:
👍 that is pretty fancy
##########
datafusion/core/src/physical_optimizer/sort_enforcement.rs:
##########
@@ -897,6 +1436,332 @@ mod tests {
Ok(())
}
+ #[tokio::test]
+ async fn test_union_inputs_different_sorted3() -> Result<()> {
+ let schema = create_test_schema()?;
+
+ let source1 = parquet_exec(&schema);
+ let sort_exprs1 = vec![
+ sort_expr("nullable_col", &schema),
+ sort_expr("non_nullable_col", &schema),
+ ];
+ let sort1 = sort_exec(sort_exprs1, source1.clone());
+ let sort_exprs2 = vec![sort_expr("nullable_col", &schema)];
+ let sort2 = sort_exec(sort_exprs2, source1);
+
+ let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)];
+ let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone());
+
+ let union = union_exec(vec![sort1, source2, sort2]);
+ let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs,
union);
+
+ // First input to the union is not Sorted (SortExec is finer than
required ordering by the SortPreservingMergeExec above).
+ // Second input to the union is already Sorted (matches with the
required ordering by the SortPreservingMergeExec above).
+ // Third input to the union is not Sorted (SortExec is matches
required ordering by the SortPreservingMergeExec above).
+ let expected_input = vec![
+ "SortPreservingMergeExec: [nullable_col@0 ASC]",
+ " UnionExec",
+ " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
projection=[nullable_col, non_nullable_col]",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
output_ordering=[nullable_col@0 ASC], projection=[nullable_col,
non_nullable_col]",
+ " SortExec: [nullable_col@0 ASC]",
+ " ParquetExec: limit=None, partitions={1 group: [[x]]},
projection=[nullable_col, non_nullable_col]",
+ ];
+ // should adjust sorting in the first input of the union such that it
is not unnecessarily fine
+ let expected_optimized = vec![
+ "SortPreservingMergeExec: [nullable_col@0 ASC]",
+ " UnionExec",
Review Comment:
that is a pretty clever plan
--
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]